diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java index 1b3a226..2f69c8d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java @@ -233,7 +233,7 @@ private void populateQueryPlan() throws IOException { mapTask.setTaskId(stage.getStageId() + "_MAP"); mapTask.setTaskType(TaskType.MAP); stage.addToTaskList(mapTask); - populateOperatorGraph(mapTask, mrTask.getWork().getAliasToWork() + populateOperatorGraph(mapTask, mrTask.getWork().getMapWork().getAliasToWork() .values()); // populate reduce task @@ -245,7 +245,7 @@ private void populateQueryPlan() throws IOException { stage.addToTaskList(reduceTask); Collection> reducerTopOps = new ArrayList>(); - reducerTopOps.add(mrTask.getWork().getReducer()); + reducerTopOps.add(mrTask.getWork().getReduceWork().getReducer()); populateOperatorGraph(reduceTask, reducerTopOps); } } else { @@ -382,7 +382,7 @@ private void extractCounters() throws IOException { } if (task instanceof ExecDriver) { ExecDriver mrTask = (ExecDriver) task; - extractOperatorCounters(mrTask.getWork().getAliasToWork().values(), + extractOperatorCounters(mrTask.getWork().getMapWork().getAliasToWork().values(), task.getId() + "_MAP"); if (mrTask.mapStarted()) { started.add(task.getId() + "_MAP"); @@ -393,7 +393,7 @@ private void extractCounters() throws IOException { if (mrTask.hasReduce()) { Collection> reducerTopOps = new ArrayList>(); - reducerTopOps.add(mrTask.getWork().getReducer()); + reducerTopOps.add(mrTask.getWork().getReduceWork().getReducer()); extractOperatorCounters(reducerTopOps, task.getId() + "_REDUCE"); if (mrTask.reduceStarted()) { started.add(task.getId() + "_REDUCE"); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index aa1035a..5e85d61 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -121,8 +121,9 @@ public JSONObject getJSONLogicalPlan(PrintStream out, ExplainWork work) throws E } if (work.getParseContext() != null) { - JSONObject jsonPlan = outputMap(work.getParseContext().getTopOps(), - "LOGICAL PLAN", out, jsonOutput, work.getExtended(), 0); + out.print("LOGICAL PLAN"); + JSONObject jsonPlan = outputMap(work.getParseContext().getTopOps(), true, + out, jsonOutput, work.getExtended(), 0); if (out != null) { out.println(); } @@ -228,19 +229,16 @@ private String indentString(int indent) { return sb.toString(); } - private JSONObject outputMap(Map mp, String header, PrintStream out, + private JSONObject outputMap(Map mp, boolean hasHeader, PrintStream out, boolean extended, boolean jsonOutput, int indent) throws Exception { - boolean first_el = true; TreeMap tree = new TreeMap(); tree.putAll(mp); JSONObject json = jsonOutput ? new JSONObject() : null; + if (out != null && hasHeader && !mp.isEmpty()) { + out.println(); + } for (Entry ent : tree.entrySet()) { - if (first_el && (out != null)) { - out.println(header); - } - first_el = false; - // Print the key if (out != null) { out.print(indentString(indent)); @@ -286,7 +284,7 @@ else if (ent.getValue() instanceof Serializable) { return jsonOutput ? json : null; } - private JSONArray outputList(List l, String header, PrintStream out, + private JSONArray outputList(List l, PrintStream out, boolean hasHeader, boolean extended, boolean jsonOutput, int indent) throws Exception { boolean first_el = true; @@ -294,10 +292,6 @@ private JSONArray outputList(List l, String header, PrintStream out, JSONArray outputArray = new JSONArray(); for (Object o : l) { - if (first_el && (out != null)) { - out.print(header); - } - if (isPrintable(o)) { String delim = first_el ? " " : ", "; if (out != null) { @@ -311,11 +305,11 @@ private JSONArray outputList(List l, String header, PrintStream out, nl = true; } else if (o instanceof Serializable) { - if (first_el && (out != null)) { + if (first_el && (out != null) && hasHeader) { out.println(); } JSONObject jsonOut = outputPlan((Serializable) o, out, extended, - jsonOutput, jsonOutput ? 0 : indent + 2); + jsonOutput, jsonOutput ? 0 : (hasHeader ? indent + 2 : indent)); if (jsonOutput) { outputArray.put(jsonOut); } @@ -433,10 +427,14 @@ private JSONObject outputPlan(Serializable work, PrintStream out, } String header = null; + boolean skipHeader = xpl_note.skipHeader(); + boolean emptyHeader = false; + if (!xpl_note.displayName().equals("")) { header = indentString(prop_indents) + xpl_note.displayName() + ":"; } else { + emptyHeader = true; prop_indents = indent; header = indentString(prop_indents); } @@ -444,7 +442,9 @@ private JSONObject outputPlan(Serializable work, PrintStream out, // Try the output as a primitive object if (isPrintable(val)) { if (out != null && shouldPrint(xpl_note, val)) { - out.printf("%s ", header); + if (!skipHeader) { + out.printf("%s ", header); + } out.println(val); } if (jsonOutput) { @@ -452,12 +452,26 @@ private JSONObject outputPlan(Serializable work, PrintStream out, } continue; } + + int ind = 0; + if (!jsonOutput) { + if (!skipHeader) { + ind = prop_indents + 2; + } else { + ind = indent; + } + } + // Try this as a map try { // Go through the map and print out the stuff Map mp = (Map) val; - JSONObject jsonOut = outputMap(mp, header, out, extended, jsonOutput, - jsonOutput ? 0 : prop_indents + 2); + + if (out != null && !skipHeader && mp != null && !mp.isEmpty()) { + out.print(header); + } + + JSONObject jsonOut = outputMap(mp, !skipHeader && !emptyHeader, out, extended, jsonOutput, ind); if (jsonOutput) { json.put(header, jsonOut); } @@ -470,8 +484,12 @@ private JSONObject outputPlan(Serializable work, PrintStream out, // Try this as a list try { List l = (List) val; - JSONArray jsonOut = outputList(l, header, out, extended, jsonOutput, - jsonOutput ? 0 : prop_indents + 2); + + if (out != null && !skipHeader && l != null && !l.isEmpty()) { + out.print(header); + } + + JSONArray jsonOut = outputList(l, out, !skipHeader && !emptyHeader, extended, jsonOutput, ind); if (jsonOutput) { json.put(header, jsonOut); @@ -486,11 +504,11 @@ private JSONObject outputPlan(Serializable work, PrintStream out, // Finally check if it is serializable try { Serializable s = (Serializable) val; - if (out != null) { + + if (!skipHeader && out != null) { out.println(header); } - JSONObject jsonOut = outputPlan(s, out, extended, jsonOutput, - jsonOutput ? 0 : prop_indents + 2); + JSONObject jsonOut = outputPlan(s, out, extended, jsonOutput, ind); if (jsonOutput) { json.put(header, jsonOut); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java index cf8bd9d..9bc8dac 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; -import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PartitionDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; @@ -63,7 +63,7 @@ * different from regular operators in that it starts off by processing a * Writable data structure from a Table (instead of a Hive Object). **/ -public class MapOperator extends Operator implements Serializable, Cloneable { +public class MapOperator extends Operator implements Serializable, Cloneable { private static final long serialVersionUID = 1L; @@ -229,14 +229,14 @@ public Converter getPartTblObjectInspectorConverter() { * @param mrwork * @throws HiveException */ - public void initializeAsRoot(Configuration hconf, MapredWork mrwork) + public void initializeAsRoot(Configuration hconf, MapWork mapWork) throws HiveException { - setConf(mrwork); + setConf(mapWork); setChildren(hconf); initialize(hconf, null); } - private MapOpCtx initObjectInspector(MapredWork conf, + private MapOpCtx initObjectInspector(MapWork conf, Configuration hconf, String onefile, Map convertedOI) throws HiveException, ClassNotFoundException, InstantiationException, IllegalAccessException, diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index a271279..cee95fd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -59,6 +59,7 @@ import org.apache.hadoop.hive.ql.plan.LoadFileDesc; import org.apache.hadoop.hive.ql.plan.LoadMultiFilesDesc; import org.apache.hadoop.hive.ql.plan.LoadTableDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.api.StageType; @@ -306,9 +307,13 @@ public int execute(DriverContext driverContext) { // the directory this move task is moving if (task instanceof MapRedTask) { MapredWork work = (MapredWork)task.getWork(); - bucketCols = work.getBucketedColsByDirectory().get(path); - sortCols = work.getSortedColsByDirectory().get(path); - numBuckets = work.getNumReduceTasks(); + MapWork mapWork = work.getMapWork(); + bucketCols = mapWork.getBucketedColsByDirectory().get(path); + sortCols = mapWork.getSortedColsByDirectory().get(path); + if (work.getReduceWork() != null) { + numBuckets = work.getReduceWork().getNumReduceTasks(); + } + if (bucketCols != null || sortCols != null) { // This must be a final map reduce task (the task containing the file sink // operator that writes the final output) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index b789d78..1ff3da6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -120,12 +120,13 @@ import org.apache.hadoop.hive.ql.plan.GroupByDesc; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.plan.PlanUtils.ExpressionTypes; +import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.api.Adjacency; import org.apache.hadoop.hive.ql.plan.api.Graph; -import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.stats.StatsFactory; import org.apache.hadoop.hive.ql.stats.StatsPublisher; @@ -223,7 +224,6 @@ public static MapredWork getMapRedWork(Configuration job) { InputStream in = new FileInputStream(path); MapredWork ret = deserializeMapRedWork(in, job); gWork = ret; - gWork.initialize(); gWorkMap.put(jobID, gWork); } return (gWork); @@ -236,15 +236,18 @@ public static MapredWork getMapRedWork(Configuration job) { public static void setWorkflowAdjacencies(Configuration conf, QueryPlan plan) { try { Graph stageGraph = plan.getQueryPlan().getStageGraph(); - if (stageGraph == null) + if (stageGraph == null) { return; + } List adjList = stageGraph.getAdjacencyList(); - if (adjList == null) + if (adjList == null) { return; + } for (Adjacency adj : adjList) { List children = adj.getChildren(); - if (children == null || children.isEmpty()) + if (children == null || children.isEmpty()) { return; + } conf.setStrings("mapreduce.workflow.adjacency."+adj.getNode(), children.toArray(new String[children.size()])); } @@ -393,7 +396,6 @@ public static void setMapRedWork(Configuration job, MapredWork w, String hiveScr } // Cache the plan in this process - w.initialize(); gWorkMap.put(jobID, w); } catch (Exception e) { e.printStackTrace(); @@ -1753,7 +1755,7 @@ public static void copyTableJobPropertiesToConf(TableDesc tbl, JobConf job) { * @return the summary of all the input paths. * @throws IOException */ - public static ContentSummary getInputSummary(Context ctx, MapredWork work, PathFilter filter) + public static ContentSummary getInputSummary(Context ctx, MapWork work, PathFilter filter) throws IOException { long[] summary = {0, 0, 0}; @@ -2214,7 +2216,7 @@ public static void reworkMapRedWork(Task task, try { MapredWork mapredWork = ((MapRedTask) task).getWork(); Set> reworkInputFormats = new HashSet>(); - for (PartitionDesc part : mapredWork.getPathToPartitionInfo().values()) { + for (PartitionDesc part : mapredWork.getMapWork().getPathToPartitionInfo().values()) { Class inputFormatCls = part .getInputFileFormatClass(); if (ReworkMapredInputFormat.class.isAssignableFrom(inputFormatCls)) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java index 9fbabae..0594826 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java @@ -56,6 +56,7 @@ import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.exec.FetchOperator; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; import org.apache.hadoop.hive.ql.exec.HiveTotalOrderPartitioner; import org.apache.hadoop.hive.ql.exec.JobCloseFeedBack; import org.apache.hadoop.hive.ql.exec.Operator; @@ -63,7 +64,6 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; import org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveKey; import org.apache.hadoop.hive.ql.io.HiveOutputFormat; @@ -73,10 +73,12 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.session.SessionState; @@ -196,13 +198,13 @@ public ExecDriver(MapredWork plan, JobConf job, boolean isSilent) throws HiveExc * @return true if fatal errors happened during job execution, false otherwise. */ public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { - for (Operator op : work.getAliasToWork().values()) { + for (Operator op : work.getMapWork().getAliasToWork().values()) { if (op.checkFatalErrors(ctrs, errMsg)) { return true; } } - if (work.getReducer() != null) { - if (work.getReducer().checkFatalErrors(ctrs, errMsg)) { + if (work.getReduceWork() != null) { + if (work.getReduceWork().getReducer().checkFatalErrors(ctrs, errMsg)) { return true; } } @@ -211,18 +213,18 @@ public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { protected void createTmpDirs() throws IOException { // fix up outputs - Map> pa = work.getPathToAliases(); + Map> pa = work.getMapWork().getPathToAliases(); if (pa != null) { List> opList = new ArrayList>(); - if (work.getReducer() != null) { - opList.add(work.getReducer()); + if (work.getReduceWork() != null) { + opList.add(work.getReduceWork().getReducer()); } for (List ls : pa.values()) { for (String a : ls) { - opList.add(work.getAliasToWork().get(a)); + opList.add(work.getMapWork().getAliasToWork().get(a)); while (!opList.isEmpty()) { Operator op = opList.remove(0); @@ -251,6 +253,7 @@ protected void createTmpDirs() throws IOException { /** * Execute a query plan using Hadoop. */ + @SuppressWarnings("deprecation") @Override public int execute(DriverContext driverContext) { @@ -259,16 +262,14 @@ public int execute(DriverContext driverContext) { boolean success = true; - String invalidReason = work.isInvalid(); - if (invalidReason != null) { - throw new RuntimeException("Plan invalid, Reason: " + invalidReason); - } - Context ctx = driverContext.getCtx(); boolean ctxCreated = false; String emptyScratchDirStr; Path emptyScratchDir; + MapWork mWork = work.getMapWork(); + ReduceWork rWork = work.getReduceWork(); + try { if (ctx == null) { ctx = new Context(job); @@ -301,27 +302,27 @@ public int execute(DriverContext driverContext) { throw new RuntimeException(e.getMessage()); } - if (work.getNumMapTasks() != null) { - job.setNumMapTasks(work.getNumMapTasks().intValue()); + if (mWork.getNumMapTasks() != null) { + job.setNumMapTasks(mWork.getNumMapTasks().intValue()); } - if (work.getMaxSplitSize() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, work.getMaxSplitSize().longValue()); + if (mWork.getMaxSplitSize() != null) { + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, mWork.getMaxSplitSize().longValue()); } - if (work.getMinSplitSize() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, work.getMinSplitSize().longValue()); + if (mWork.getMinSplitSize() != null) { + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, mWork.getMinSplitSize().longValue()); } - if (work.getMinSplitSizePerNode() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, work.getMinSplitSizePerNode().longValue()); + if (mWork.getMinSplitSizePerNode() != null) { + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, mWork.getMinSplitSizePerNode().longValue()); } - if (work.getMinSplitSizePerRack() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, work.getMinSplitSizePerRack().longValue()); + if (mWork.getMinSplitSizePerRack() != null) { + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, mWork.getMinSplitSizePerRack().longValue()); } - job.setNumReduceTasks(work.getNumReduceTasks().intValue()); + job.setNumReduceTasks(rWork != null ? rWork.getNumReduceTasks().intValue() : 0); job.setReducerClass(ExecReducer.class); // set input format information if necessary @@ -338,7 +339,7 @@ public int execute(DriverContext driverContext) { inpFormat = ShimLoader.getHadoopShims().getInputFormatClassName(); } - if (getWork().isUseBucketizedHiveInputFormat()) { + if (mWork.isUseBucketizedHiveInputFormat()) { inpFormat = BucketizedHiveInputFormat.class.getName(); } @@ -387,11 +388,11 @@ public int execute(DriverContext driverContext) { } try{ - MapredLocalWork localwork = work.getMapLocalWork(); + MapredLocalWork localwork = mWork.getMapLocalWork(); if (localwork != null) { if (!ShimLoader.getHadoopShims().isLocalMode(job)) { Path localPath = new Path(localwork.getTmpFileURI()); - Path hdfsPath = new Path(work.getTmpHDFSFileURI()); + Path hdfsPath = new Path(mWork.getTmpHDFSFileURI()); FileSystem hdfs = hdfsPath.getFileSystem(job); FileSystem localFS = localPath.getFileSystem(job); @@ -429,17 +430,17 @@ public int execute(DriverContext driverContext) { } } work.configureJobConf(job); - addInputPaths(job, work, emptyScratchDirStr, ctx); + addInputPaths(job, mWork, emptyScratchDirStr, ctx); Utilities.setMapRedWork(job, work, ctx.getMRTmpFileURI()); - if (work.getSamplingType() > 0 && work.getNumReduceTasks() > 1) { + if (mWork.getSamplingType() > 0 && rWork != null && rWork.getNumReduceTasks() > 1) { try { - handleSampling(driverContext, work, job, new HiveConf(conf)); + handleSampling(driverContext, mWork, job, new HiveConf(conf)); job.setPartitionerClass(HiveTotalOrderPartitioner.class); } catch (Exception e) { console.printInfo("Not enough sampling data.. Rolling back to single reducer task"); - work.setNumReduceTasks(1); + rWork.setNumReduceTasks(1); job.setNumReduceTasks(1); } } @@ -454,7 +455,7 @@ public int execute(DriverContext driverContext) { // make this client wait if job trcker is not behaving well. Throttle.checkJobTracker(job, LOG); - if (work.isGatheringStats()) { + if (mWork.isGatheringStats() || (rWork != null && rWork.isGatheringStats())) { // initialize stats publishing table StatsPublisher statsPublisher; String statsImplementationClass = HiveConf.getVar(job, HiveConf.ConfVars.HIVESTATSDBCLASS); @@ -517,13 +518,13 @@ public int execute(DriverContext driverContext) { try { if (rj != null) { JobCloseFeedBack feedBack = new JobCloseFeedBack(); - if (work.getAliasToWork() != null) { - for (Operator op : work.getAliasToWork().values()) { + if (mWork.getAliasToWork() != null) { + for (Operator op : mWork.getAliasToWork().values()) { op.jobClose(job, success, feedBack); } } - if (work.getReducer() != null) { - work.getReducer().jobClose(job, success, feedBack); + if (rWork != null) { + rWork.getReducer().jobClose(job, success, feedBack); } } } catch (Exception e) { @@ -539,16 +540,16 @@ public int execute(DriverContext driverContext) { return (returnVal); } - private void handleSampling(DriverContext context, MapredWork work, JobConf job, HiveConf conf) + private void handleSampling(DriverContext context, MapWork mWork, JobConf job, HiveConf conf) throws Exception { - assert work.getAliasToWork().keySet().size() == 1; + assert mWork.getAliasToWork().keySet().size() == 1; - String alias = work.getAliases().get(0); - Operator topOp = work.getAliasToWork().get(alias); - PartitionDesc partDesc = work.getAliasToPartnInfo().get(alias); + String alias = mWork.getAliases().get(0); + Operator topOp = mWork.getAliasToWork().get(alias); + PartitionDesc partDesc = mWork.getAliasToPartnInfo().get(alias); - ArrayList paths = work.getPaths(); - ArrayList parts = work.getPartitionDescs(); + ArrayList paths = mWork.getPaths(); + ArrayList parts = mWork.getPartitionDescs(); Path onePath = new Path(paths.get(0)); String tmpPath = context.getCtx().getExternalTmpFileURI(onePath.toUri()); @@ -558,7 +559,7 @@ private void handleSampling(DriverContext context, MapredWork work, JobConf job, PartitionKeySampler sampler = new PartitionKeySampler(); - if (work.getSamplingType() == MapredWork.SAMPLING_ON_PREV_MR) { + if (mWork.getSamplingType() == MapWork.SAMPLING_ON_PREV_MR) { console.printInfo("Use sampling data created in previous MR"); // merges sampling data from previous MR and make paritition keys for total sort for (String path : paths) { @@ -568,7 +569,7 @@ private void handleSampling(DriverContext context, MapredWork work, JobConf job, sampler.addSampleFile(status.getPath(), job); } } - } else if (work.getSamplingType() == MapredWork.SAMPLING_ON_START) { + } else if (mWork.getSamplingType() == MapWork.SAMPLING_ON_START) { console.printInfo("Creating sampling data.."); assert topOp instanceof TableScanOperator; TableScanOperator ts = (TableScanOperator) topOp; @@ -592,7 +593,7 @@ private void handleSampling(DriverContext context, MapredWork work, JobConf job, fetcher.clearFetchContext(); } } else { - throw new IllegalArgumentException("Invalid sampling type " + work.getSamplingType()); + throw new IllegalArgumentException("Invalid sampling type " + mWork.getSamplingType()); } sampler.writePartitionKeys(partitionFile, job); } @@ -601,16 +602,17 @@ private void handleSampling(DriverContext context, MapredWork work, JobConf job, * Set hive input format, and input format file if necessary. */ protected void setInputAttributes(Configuration conf) { - if (work.getInputformat() != null) { - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEINPUTFORMAT, work.getInputformat()); + MapWork mWork = work.getMapWork(); + if (mWork.getInputformat() != null) { + HiveConf.setVar(conf, HiveConf.ConfVars.HIVEINPUTFORMAT, mWork.getInputformat()); } - if (work.getIndexIntermediateFile() != null) { - conf.set("hive.index.compact.file", work.getIndexIntermediateFile()); - conf.set("hive.index.blockfilter.file", work.getIndexIntermediateFile()); + if (mWork.getIndexIntermediateFile() != null) { + conf.set("hive.index.compact.file", mWork.getIndexIntermediateFile()); + conf.set("hive.index.blockfilter.file", mWork.getIndexIntermediateFile()); } // Intentionally overwrites anything the user may have put here - conf.setBoolean("hive.input.format.sorted", work.isInputFormatSorted()); + conf.setBoolean("hive.input.format.sorted", mWork.isInputFormatSorted()); } public boolean mapStarted() { @@ -823,19 +825,19 @@ public boolean isMapRedTask() { @Override public Collection> getTopOperators() { - return getWork().getAliasToWork().values(); + return getWork().getMapWork().getAliasToWork().values(); } @Override public boolean hasReduce() { MapredWork w = getWork(); - return w.getReducer() != null; + return w.getReduceWork() != null; } /** * Handle a empty/null path for a given alias. */ - private static int addInputPath(String path, JobConf job, MapredWork work, String hiveScratchDir, + private static int addInputPath(String path, JobConf job, MapWork work, String hiveScratchDir, int numEmptyPaths, boolean isEmptyPath, String alias) throws Exception { // either the directory does not exist or it is empty assert path == null || isEmptyPath; @@ -919,7 +921,7 @@ private static int addInputPath(String path, JobConf job, MapredWork work, Strin return numEmptyPaths; } - public static void addInputPaths(JobConf job, MapredWork work, String hiveScratchDir, Context ctx) + public static void addInputPaths(JobConf job, MapWork work, String hiveScratchDir, Context ctx) throws Exception { int numEmptyPaths = 0; @@ -1002,11 +1004,11 @@ public String getName() { @Override public void updateCounters(Counters ctrs, RunningJob rj) throws IOException { - for (Operator op : work.getAliasToWork().values()) { + for (Operator op : work.getMapWork().getAliasToWork().values()) { op.updateCounters(ctrs); } - if (work.getReducer() != null) { - work.getReducer().updateCounters(ctrs); + if (work.getReduceWork() != null) { + work.getReduceWork().getReducer().updateCounters(ctrs); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java index d48a47f..9452e8a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java @@ -98,12 +98,12 @@ public void configure(JobConf job) { // create map and fetch operators MapredWork mrwork = Utilities.getMapRedWork(job); mo = new MapOperator(); - mo.setConf(mrwork); + mo.setConf(mrwork.getMapWork()); // initialize map operator mo.setChildren(job); l4j.info(mo.dump(0)); // initialize map local work - localWork = mrwork.getMapLocalWork(); + localWork = mrwork.getMapWork().getMapLocalWork(); execContext.setLocalWork(localWork); UDFContext.init(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java index 0cbedfc..ea46523 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.mr.ExecMapper.reportStats; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.SerDe; @@ -121,7 +121,7 @@ public void configure(JobConf job) { l4j.info("cannot get classpath: " + e.getMessage()); } jc = job; - MapredWork gWork = Utilities.getMapRedWork(job); + ReduceWork gWork = Utilities.getMapRedWork(job).getReduceWork(); reducer = gWork.getReducer(); reducer.setParentOperators(null); // clear out any parents as reducer is the // root diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java index 9676e7e..8cb283d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java @@ -40,8 +40,10 @@ import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.mapred.JobConf; @@ -101,7 +103,7 @@ public int execute(DriverContext driverContext) { conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO)) { if (inputSummary == null) { - inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work, null); + inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work.getMapWork(), null); } // set the values of totalInputFileSize and totalInputNumFiles, estimating them @@ -109,7 +111,7 @@ public int execute(DriverContext driverContext) { estimateInputSize(); // at this point the number of reducers is precisely defined in the plan - int numReducers = work.getNumReduceTasks(); + int numReducers = work.getReduceWork() == null ? 0 : work.getReduceWork().getNumReduceTasks(); if (LOG.isDebugEnabled()) { LOG.debug("Task: " + getId() + ", Summary: " + @@ -383,26 +385,26 @@ public boolean reduceDone() { * Set the number of reducers for the mapred work. */ private void setNumberOfReducers() throws IOException { + ReduceWork rWork = work.getReduceWork(); // this is a temporary hack to fix things that are not fixed in the compiler - Integer numReducersFromWork = work.getNumReduceTasks(); + Integer numReducersFromWork = rWork == null ? 0 : rWork.getNumReduceTasks(); - if (work.getReducer() == null) { + if (rWork == null) { console .printInfo("Number of reduce tasks is set to 0 since there's no reduce operator"); - work.setNumReduceTasks(Integer.valueOf(0)); } else { if (numReducersFromWork >= 0) { console.printInfo("Number of reduce tasks determined at compile time: " - + work.getNumReduceTasks()); + + rWork.getNumReduceTasks()); } else if (job.getNumReduceTasks() > 0) { int reducers = job.getNumReduceTasks(); - work.setNumReduceTasks(reducers); + rWork.setNumReduceTasks(reducers); console .printInfo("Number of reduce tasks not specified. Defaulting to jobconf value of: " + reducers); } else { int reducers = estimateNumberOfReducers(); - work.setNumReduceTasks(reducers); + rWork.setNumReduceTasks(reducers); console .printInfo("Number of reduce tasks not specified. Estimated from input data size: " + reducers); @@ -437,7 +439,7 @@ private int estimateNumberOfReducers() throws IOException { if(inputSummary == null) { // compute the summary and stash it away - inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work, null); + inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work.getMapWork(), null); } // if all inputs are sampled, we should shrink the size of reducers accordingly. @@ -459,7 +461,7 @@ private int estimateNumberOfReducers() throws IOException { // and the user has configured Hive to do this, make sure the number of reducers is a // power of two if (conf.getBoolVar(HiveConf.ConfVars.HIVE_INFER_BUCKET_SORT_NUM_BUCKETS_POWER_TWO) && - work.isFinalMapRed() && !work.getBucketedColsByDirectory().isEmpty()) { + work.isFinalMapRed() && !work.getMapWork().getBucketedColsByDirectory().isEmpty()) { int reducersLog = (int)(Math.log(reducers) / Math.log(2)) + 1; int reducersPowerTwo = (int)Math.pow(2, reducersLog); @@ -497,11 +499,13 @@ private void estimateInputSize() { return; } + MapWork mWork = work.getMapWork(); + // Initialize the values to be those taken from the input summary totalInputFileSize = inputSummary.getLength(); totalInputNumFiles = inputSummary.getFileCount(); - if (work.getNameToSplitSample() == null || work.getNameToSplitSample().isEmpty()) { + if (mWork.getNameToSplitSample() == null || mWork.getNameToSplitSample().isEmpty()) { // If percentage block sampling wasn't used, we don't need to do any estimation inputSizeEstimated = true; return; @@ -510,10 +514,10 @@ private void estimateInputSize() { // if all inputs are sampled, we should shrink the size of the input accordingly double highestSamplePercentage = 0; boolean allSample = false; - for (String alias : work.getAliasToWork().keySet()) { - if (work.getNameToSplitSample().containsKey(alias)) { + for (String alias : mWork.getAliasToWork().keySet()) { + if (mWork.getNameToSplitSample().containsKey(alias)) { allSample = true; - Double rate = work.getNameToSplitSample().get(alias).getPercent(); + Double rate = mWork.getNameToSplitSample().get(alias).getPercent(); if (rate != null && rate > highestSamplePercentage) { highestSamplePercentage = rate; } @@ -580,7 +584,7 @@ public static String isEligibleForLocalMode(HiveConf conf, @Override public Operator getReducer() { - return getWork().getReducer(); + return getWork().getReduceWork() == null ? null : getWork().getReduceWork().getReducer(); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java index 81f7a99..af4e208 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PartitionDesc; @@ -188,7 +189,7 @@ public void generateIndexQuery(List indexes, ExprNodeDesc predicate, if (pctx.getConf().getBoolVar(ConfVars.HIVE_INDEX_COMPACT_BINARY_SEARCH) && useSorted) { // For now, only works if the predicate is a single condition - MapredWork work = null; + MapWork work = null; String originalInputFormat = null; for (Task task : driver.getPlan().getRootTasks()) { // The index query should have one and only one map reduce task in the root tasks @@ -202,7 +203,9 @@ public void generateIndexQuery(List indexes, ExprNodeDesc predicate, work.setInputFormatSorted(false); break; } - work = (MapredWork)task.getWork(); + if (task.getWork() != null) { + work = ((MapredWork)task.getWork()).getMapWork(); + } String inputFormat = work.getInputformat(); originalInputFormat = inputFormat; if (inputFormat == null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java index 9ab4b24..69b680d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java @@ -95,7 +95,7 @@ public CombineHiveInputSplit(JobConf job, InputSplitShim inputSplitShim) this.inputSplitShim = inputSplitShim; if (job != null) { Map pathToPartitionInfo = Utilities - .getMapRedWork(job).getPathToPartitionInfo(); + .getMapRedWork(job).getMapWork().getPathToPartitionInfo(); // extract all the inputFormatClass names for each chunk in the // CombinedSplit. @@ -200,7 +200,7 @@ public void write(DataOutput out) throws IOException { if (inputFormatClassName == null) { Map pathToPartitionInfo = Utilities - .getMapRedWork(getJob()).getPathToPartitionInfo(); + .getMapRedWork(getJob()).getMapWork().getPathToPartitionInfo(); // extract all the inputFormatClass names for each chunk in the // CombinedSplit. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java index adf4923..8cc5442 100755 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PartitionDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; @@ -249,10 +249,10 @@ public RecordReader getRecordReader(InputSplit split, JobConf job, } protected Map pathToPartitionInfo; - MapredWork mrwork = null; + MapWork mrwork = null; protected void init(JobConf job) { - mrwork = Utilities.getMapRedWork(job); + mrwork = Utilities.getMapRedWork(job).getMapWork(); pathToPartitionInfo = mrwork.getPathToPartitionInfo(); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/SymbolicInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/SymbolicInputFormat.java index 9ae58f4..37e3879 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/SymbolicInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/SymbolicInputFormat.java @@ -37,10 +37,10 @@ public class SymbolicInputFormat implements ReworkMapredInputFormat { public void rework(HiveConf job, MapredWork work) throws IOException { - Map pathToParts = work.getPathToPartitionInfo(); + Map pathToParts = work.getMapWork().getPathToPartitionInfo(); List toRemovePaths = new ArrayList(); Map toAddPathToPart = new HashMap(); - Map> pathToAliases = work.getPathToAliases(); + Map> pathToAliases = work.getMapWork().getPathToAliases(); for (Map.Entry pathPartEntry : pathToParts .entrySet()) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java index dbc999f..54836c9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java @@ -18,6 +18,10 @@ package org.apache.hadoop.hive.ql.io.avro; +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; import org.apache.avro.generic.GenericData; @@ -35,12 +39,11 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerdeException; import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.*; - - -import java.io.IOException; -import java.util.Map; -import java.util.Properties; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; /** * RecordReader optimized against Avro GenericRecords that returns to record @@ -67,7 +70,9 @@ public AvroGenericRecordReader(JobConf job, FileSplit split, Reporter reporter) GenericDatumReader gdr = new GenericDatumReader(); - if(latest != null) gdr.setExpected(latest); + if(latest != null) { + gdr.setExpected(latest); + } this.reader = new DataFileReader(new FsInput(split.getPath(), job), gdr); this.reader.sync(split.getStart()); @@ -90,7 +95,7 @@ private Schema getSchema(JobConf job, FileSplit split) throws AvroSerdeException // Iterate over the Path -> Partition descriptions to find the partition // that matches our input split. - for (Map.Entry pathsAndParts: mapRedWork.getPathToPartitionInfo().entrySet()){ + for (Map.Entry pathsAndParts: mapRedWork.getMapWork().getPathToPartitionInfo().entrySet()){ String partitionPath = pathsAndParts.getKey(); if(pathIsInPartition(split.getPath(), partitionPath)) { if(LOG.isInfoEnabled()) { @@ -101,11 +106,15 @@ private Schema getSchema(JobConf job, FileSplit split) throws AvroSerdeException Properties props = pathsAndParts.getValue().getProperties(); if(props.containsKey(AvroSerdeUtils.SCHEMA_LITERAL) || props.containsKey(AvroSerdeUtils.SCHEMA_URL)) { return AvroSerdeUtils.determineSchemaOrThrowException(props); - } else + } + else { return null; // If it's not in this property, it won't be in any others + } } } - if(LOG.isInfoEnabled()) LOG.info("Unable to match filesplit " + split + " with a partition."); + if(LOG.isInfoEnabled()) { + LOG.info("Unable to match filesplit " + split + " with a partition."); + } } // In "select * from table" situations (non-MR), we can add things to the job diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/BlockMergeTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/BlockMergeTask.java index ad14966..6ef9f98 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/BlockMergeTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/BlockMergeTask.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.exec.mr.Throttle; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormatImpl; +import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; @@ -192,7 +193,9 @@ public int execute(DriverContext driverContext) { try { addInputPaths(job, work); - Utilities.setMapRedWork(job, work, ctx.getMRTmpFileURI()); + MapredWork mrWork = new MapredWork(); + mrWork.setMapWork(work); + Utilities.setMapRedWork(job, mrWork, ctx.getMRTmpFileURI()); // remove the pwd from conf file so that job tracker doesn't show this // logs diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/MergeWork.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/MergeWork.java index 16f45ec..bc306c1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/MergeWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/MergeWork.java @@ -33,14 +33,14 @@ import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; import org.apache.hadoop.hive.ql.plan.Explain; import org.apache.hadoop.hive.ql.plan.ListBucketingCtx; -import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.Mapper; @Explain(displayName = "Block level merge") -public class MergeWork extends MapredWork implements Serializable { +public class MergeWork extends MapWork implements Serializable { private static final long serialVersionUID = 1L; @@ -70,9 +70,6 @@ public MergeWork(List inputPaths, String outputDir, if(this.getPathToPartitionInfo() == null) { this.setPathToPartitionInfo(new LinkedHashMap()); } - if(this.getNumReduceTasks() == null) { - this.setNumReduceTasks(0); - } for(String path: this.inputPaths) { this.getPathToPartitionInfo().put(path, partDesc); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java index f66b82e..403ee10 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormatImpl; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; @@ -181,7 +182,9 @@ public int execute(DriverContext driverContext) { try { addInputPaths(job, work); - Utilities.setMapRedWork(job, work, ctx.getMRTmpFileURI()); + MapredWork mrWork = new MapredWork(); + mrWork.setMapWork(work); + Utilities.setMapRedWork(job, mrWork, ctx.getMRTmpFileURI()); // remove the pwd from conf file so that job tracker doesn't show this // logs diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanWork.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanWork.java index 4a1a67e..4eb86ba 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanWork.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; import org.apache.hadoop.hive.ql.io.rcfile.merge.RCFileBlockMergeInputFormat; import org.apache.hadoop.hive.ql.plan.Explain; -import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; import org.apache.hadoop.mapred.Mapper; @@ -34,7 +34,7 @@ * */ @Explain(displayName = "Partial Scan Statistics") -public class PartialScanWork extends MapredWork implements Serializable { +public class PartialScanWork extends MapWork implements Serializable { private static final long serialVersionUID = 1L; @@ -52,9 +52,6 @@ public PartialScanWork(List inputPaths) { if(this.getPathToPartitionInfo() == null) { this.setPathToPartitionInfo(new LinkedHashMap()); } - if(this.getNumReduceTasks() == null) { - this.setNumReduceTasks(0); - } for(String path: this.inputPaths) { this.getPathToPartitionInfo().put(path, partDesc); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateMapper.java index 47b18ba..4cd57f8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateMapper.java @@ -76,7 +76,7 @@ public ColumnTruncateMapper() { @Override public void configure(JobConf job) { jc = job; - work = (ColumnTruncateWork) Utilities.getMapRedWork(job); + work = (ColumnTruncateWork) Utilities.getMapRedWork(job).getMapWork(); String specPath = work.getOutputDir(); Path tmpPath = Utilities.toTempPath(specPath); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java index 6beb54d..e192d7b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.exec.mr.Throttle; import org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormatImpl; +import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.shims.ShimLoader; @@ -165,7 +166,9 @@ public int execute(DriverContext driverContext) { try { addInputPaths(job, work); - Utilities.setMapRedWork(job, work, ctx.getMRTmpFileURI()); + MapredWork mrWork = new MapredWork(); + mrWork.setMapWork(work); + Utilities.setMapRedWork(job, mrWork, ctx.getMRTmpFileURI()); // remove the pwd from conf file so that job tracker doesn't show this // logs diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateWork.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateWork.java index edbb098..1a0b2ab 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateWork.java @@ -27,12 +27,12 @@ import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; import org.apache.hadoop.hive.ql.plan.Explain; import org.apache.hadoop.hive.ql.plan.ListBucketingCtx; -import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; import org.apache.hadoop.mapred.Mapper; @Explain(displayName = "Column Truncate") -public class ColumnTruncateWork extends MapredWork implements Serializable { +public class ColumnTruncateWork extends MapWork implements Serializable { private static final long serialVersionUID = 1L; @@ -64,9 +64,6 @@ public ColumnTruncateWork(List droppedColumns, String inputDir, String if(this.getPathToPartitionInfo() == null) { this.setPathToPartitionInfo(new LinkedHashMap()); } - if(this.getNumReduceTasks() == null) { - this.setNumReduceTasks(0); - } this.getPathToPartitionInfo().put(inputDir, partDesc); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java index 4163fd0..9a3a4ae 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java @@ -57,6 +57,7 @@ import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.LoadFileDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; @@ -142,10 +143,10 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx opProcCtx, // or for a map-reduce job MapredWork currWork = (MapredWork) currTask.getWork(); boolean mergeMapOnly = - hconf.getBoolVar(ConfVars.HIVEMERGEMAPFILES) && currWork.getReducer() == null; + hconf.getBoolVar(ConfVars.HIVEMERGEMAPFILES) && currWork.getReduceWork() == null; boolean mergeMapRed = hconf.getBoolVar(ConfVars.HIVEMERGEMAPREDFILES) && - currWork.getReducer() != null; + currWork.getReduceWork() != null; if (mergeMapOnly || mergeMapRed) { chDir = true; } @@ -201,7 +202,7 @@ private void processLinkedFileDesc(GenMRProcContext ctx, if (!seenOps.contains(currTopOp)) { seenOps.add(currTopOp); GenMapRedUtils.setTaskPlan(currAliasId, currTopOp, - (MapredWork) currTask.getWork(), false, ctx); + ((MapredWork) currTask.getWork()).getMapWork(), false, ctx); } if (!rootTasks.contains(currTask) @@ -251,7 +252,10 @@ private void addStatsTask(FileSinkOperator nd, MoveTask mvTask, // mark the MapredWork and FileSinkOperator for gathering stats nd.getConf().setGatherStats(true); - mrWork.setGatheringStats(true); + mrWork.getMapWork().setGatheringStats(true); + if (mrWork.getReduceWork() != null) { + mrWork.getReduceWork().setGatheringStats(true); + } nd.getConf().setStatsReliable(hconf.getBoolVar(ConfVars.HIVE_STATS_RELIABLE)); nd.getConf().setMaxStatsKeyPrefixLength( hconf.getIntVar(ConfVars.HIVE_STATS_KEY_PREFIX_MAX_LENGTH)); @@ -368,8 +372,9 @@ private void createMRWorkForMergingFiles (FileSinkOperator fsInput, GenMRProcCon Class c = (Class) Class.forName(inputFormatClass); LOG.info("RCFile format- Using block level merge"); - cplan = createRCFileMergeTask(fsInputDesc, finalName, - dpCtx != null && dpCtx.getNumDPCols() > 0); + cplan = new MapredWork(); + cplan.setMapWork(createRCFileMergeTask(fsInputDesc, finalName, + dpCtx != null && dpCtx.getNumDPCols() > 0)); } catch (ClassNotFoundException e) { String msg = "Illegal input format class: " + inputFormatClass; throw new SemanticException(msg); @@ -379,7 +384,7 @@ private void createMRWorkForMergingFiles (FileSinkOperator fsInput, GenMRProcCon cplan = createMRWorkForMergingFiles(conf, tsMerge, fsInputDesc); // use CombineHiveInputFormat for map-only merging } - cplan.setInputformat("org.apache.hadoop.hive.ql.io.CombineHiveInputFormat"); + cplan.getMapWork().setInputformat("org.apache.hadoop.hive.ql.io.CombineHiveInputFormat"); // NOTE: we should gather stats in MR1 rather than MR2 at merge job since we don't // know if merge MR2 will be triggered at execution time ConditionalTask cndTsk = createCondTask(conf, ctx.getCurrTask(), dummyMv, cplan, @@ -492,14 +497,14 @@ private MapredWork createMRWorkForMergingFiles (HiveConf conf, aliases.add(inputDir); // dummy alias: just use the input path // constructing the default MapredWork - MapredWork cplan = GenMapRedUtils.getMapRedWorkFromConf(conf); + MapredWork cMrPlan = GenMapRedUtils.getMapRedWorkFromConf(conf); + MapWork cplan = cMrPlan.getMapWork(); cplan.getPathToAliases().put(inputDir, aliases); cplan.getPathToPartitionInfo().put(inputDir, new PartitionDesc(tblDesc, null)); - cplan.setNumReduceTasks(0); cplan.getAliasToWork().put(inputDir, topOp); cplan.setMapperCannotSpanPartns(true); - return cplan; + return cMrPlan; } /** @@ -510,7 +515,7 @@ private MapredWork createMRWorkForMergingFiles (HiveConf conf, * @return MergeWork if table is stored as RCFile, * null otherwise */ - private MapredWork createRCFileMergeTask(FileSinkDesc fsInputDesc, + private MapWork createRCFileMergeTask(FileSinkDesc fsInputDesc, String finalName, boolean hasDynamicPartitions) throws SemanticException { String inputDir = fsInputDesc.getFinalDirName(); @@ -720,7 +725,7 @@ private String processFS(FileSinkOperator fsOp, Stack stack, if (!seenOps.contains(currTopOp)) { seenOps.add(currTopOp); GenMapRedUtils.setTaskPlan(currAliasId, currTopOp, - (MapredWork) currTask.getWork(), false, ctx); + ((MapredWork) currTask.getWork()).getMapWork(), false, ctx); } opTaskMap.put(null, currTask); if (!rootTasks.contains(currTask) @@ -732,7 +737,7 @@ private String processFS(FileSinkOperator fsOp, Stack stack, if (!seenOps.contains(currTopOp)) { seenOps.add(currTopOp); GenMapRedUtils.setTaskPlan(currAliasId, currTopOp, - (MapredWork) mapTask.getWork(), false, ctx); + ((MapredWork) mapTask.getWork()).getMapWork(), false, ctx); } else { UnionOperator currUnionOp = ctx.getCurrUnionOp(); if (currUnionOp != null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java index 50d645f..52237e5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java @@ -77,7 +77,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx opProcCtx, // If the plan for this reducer does not exist, initialize the plan if (opMapTask == null) { - if (currPlan.getReducer() == null) { + if (currPlan.getReduceWork() == null) { GenMapRedUtils.initPlan(op, ctx); } else { GenMapRedUtils.splitPlan(op, ctx); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java index e299a56..17e3aeb 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java @@ -85,13 +85,13 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx opProcCtx, // If the plan for this reducer does not exist, initialize the plan if (reducerTask == null) { // When the reducer is encountered for the first time - if (plan.getReducer() == null) { + if (plan.getReduceWork() == null) { GenMapRedUtils.initUnionPlan(op, union, ctx, unionTask); // When union is followed by a multi-table insert } else { GenMapRedUtils.splitPlan(op, ctx); } - } else if (plan.getReducer() == reducer) { + } else if (plan.getReduceWork().getReducer() == reducer) { // The union is already initialized. However, the union is walked from // another input // initUnionPlan is idempotent diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java index 7178b55..1965f53 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java @@ -117,7 +117,10 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx opProcCtx, handlePartialScanCommand(op, ctx, parseCtx, currTask, parseInfo, statsWork, statsTask); } - currWork.setGatheringStats(true); + currWork.getMapWork().setGatheringStats(true); + if (currWork.getReduceWork() != null) { + currWork.getReduceWork().setGatheringStats(true); + } // NOTE: here we should use the new partition predicate pushdown API to get a list of pruned list, // and pass it to setTaskPlan as the last parameter Set confirmedPartns = new HashSet(); @@ -139,9 +142,9 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx opProcCtx, Table source = parseCtx.getQB().getMetaData().getTableForAlias(alias); PrunedPartitionList partList = new PrunedPartitionList(source, confirmedPartns, new HashSet(), null); - GenMapRedUtils.setTaskPlan(currAliasId, currTopOp, currWork, false, ctx, partList); + GenMapRedUtils.setTaskPlan(currAliasId, currTopOp, currWork.getMapWork(), false, ctx, partList); } else { // non-partitioned table - GenMapRedUtils.setTaskPlan(currAliasId, currTopOp, currWork, false, ctx); + GenMapRedUtils.setTaskPlan(currAliasId, currTopOp, currWork.getMapWork(), false, ctx); } } return null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java index a719a83..9e36938 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java @@ -199,7 +199,7 @@ private void processSubQueryUnionMerge(GenMRProcContext ctx, if (!seenOps.contains(topOp) && topOp != null) { seenOps.add(topOp); GenMapRedUtils.setTaskPlan(ctx.getCurrAliasId(), ctx - .getCurrTopOp(), plan, false, ctx); + .getCurrTopOp(), plan.getMapWork(), false, ctx); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java index 01fbca5..728426b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -60,12 +60,14 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; 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.ReduceWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; @@ -101,10 +103,11 @@ public static void initPlan(ReduceSinkOperator op, GenMRProcContext opProcCtx) Operator currTopOp = opProcCtx.getCurrTopOp(); opTaskMap.put(reducer, currTask); - plan.setReducer(reducer); + plan.setReduceWork(new ReduceWork()); + plan.getReduceWork().setReducer(reducer); ReduceSinkDesc desc = op.getConf(); - plan.setNumReduceTasks(desc.getNumReducers()); + plan.getReduceWork().setNumReduceTasks(desc.getNumReducers()); List> rootTasks = opProcCtx.getRootTasks(); @@ -114,7 +117,7 @@ public static void initPlan(ReduceSinkOperator op, GenMRProcContext opProcCtx) rootTasks.add(currTask); } if (reducer.getClass() == JoinOperator.class) { - plan.setNeedsTagging(true); + plan.getReduceWork().setNeedsTagging(true); } assert currTopOp != null; @@ -123,7 +126,7 @@ public static void initPlan(ReduceSinkOperator op, GenMRProcContext opProcCtx) if (!seenOps.contains(currTopOp)) { seenOps.add(currTopOp); - setTaskPlan(currAliasId, currTopOp, plan, false, opProcCtx); + setTaskPlan(currAliasId, currTopOp, plan.getMapWork(), false, opProcCtx); } currTopOp = null; @@ -153,13 +156,16 @@ public static void initUnionPlan(ReduceSinkOperator op, UnionOperator currUnionO opProcCtx.getOpTaskMap(); opTaskMap.put(reducer, unionTask); - plan.setReducer(reducer); + + plan.setReduceWork(new ReduceWork()); + plan.getReduceWork().setReducer(reducer); + plan.getReduceWork().setReducer(reducer); ReduceSinkDesc desc = op.getConf(); - plan.setNumReduceTasks(desc.getNumReducers()); + plan.getReduceWork().setNumReduceTasks(desc.getNumReducers()); if (reducer.getClass() == JoinOperator.class) { - plan.setNeedsTagging(true); + plan.getReduceWork().setNeedsTagging(true); } initUnionPlan(opProcCtx, currUnionOp, unionTask, false); @@ -175,7 +181,7 @@ private static void setUnionPlan(GenMRProcContext opProcCtx, String currAliasId = opProcCtx.getCurrAliasId(); if (!seenOps.contains(currTopOp) || mergeTask) { seenOps.add(currTopOp); - setTaskPlan(currAliasId, currTopOp, plan, local, opProcCtx); + setTaskPlan(currAliasId, currTopOp, plan.getMapWork(), local, opProcCtx); } currTopOp = null; opProcCtx.setCurrTopOp(currTopOp); @@ -194,13 +200,14 @@ private static void setUnionPlan(GenMRProcContext opProcCtx, 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, + MapWork mWork = plan.getMapWork(); + if (mWork.getPathToAliases().get(taskTmpDir) == null) { + mWork.getPathToAliases().put(taskTmpDir, new ArrayList()); - plan.getPathToAliases().get(taskTmpDir).add(taskTmpDir); - plan.getPathToPartitionInfo().put(taskTmpDir, + mWork.getPathToAliases().get(taskTmpDir).add(taskTmpDir); + mWork.getPathToPartitionInfo().put(taskTmpDir, new PartitionDesc(tt_desc, null)); - plan.getAliasToWork().put(taskTmpDir, topOperators.get(pos)); + mWork.getAliasToWork().put(taskTmpDir, topOperators.get(pos)); } } } @@ -323,7 +330,7 @@ public static void joinPlan(Operator op, local = (pos == ((MapJoinDesc) op.getConf()).getPosBigTable()) ? false : true; } - setTaskPlan(currAliasId, currTopOp, plan, local, opProcCtx); + setTaskPlan(currAliasId, currTopOp, plan.getMapWork(), local, opProcCtx); } currTopOp = null; opProcCtx.setCurrTopOp(currTopOp); @@ -359,10 +366,12 @@ public static void splitPlan(ReduceSinkOperator op, GenMRProcContext opProcCtx) Operator reducer = op.getChildOperators().get(0); // Add the reducer - cplan.setReducer(reducer); + ReduceWork rWork = new ReduceWork(); + cplan.setReduceWork(rWork); + rWork.setReducer(reducer); ReduceSinkDesc desc = op.getConf(); - cplan.setNumReduceTasks(new Integer(desc.getNumReducers())); + cplan.getReduceWork().setNumReduceTasks(new Integer(desc.getNumReducers())); HashMap, Task> opTaskMap = opProcCtx.getOpTaskMap(); @@ -388,7 +397,7 @@ public static void splitPlan(ReduceSinkOperator op, GenMRProcContext opProcCtx) * processing context */ public static void setTaskPlan(String alias_id, - Operator topOp, MapredWork plan, boolean local, + Operator topOp, MapWork plan, boolean local, GenMRProcContext opProcCtx) throws SemanticException { setTaskPlan(alias_id, topOp, plan, local, opProcCtx, null); } @@ -432,7 +441,7 @@ private static ReadEntity getParentViewInfo(String alias_id, * 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, + Operator topOp, MapWork plan, boolean local, GenMRProcContext opProcCtx, PrunedPartitionList pList) throws SemanticException { ParseContext parseCtx = opProcCtx.getParseCtx(); Set inputs = opProcCtx.getInputs(); @@ -698,7 +707,7 @@ public static void setTaskPlan(String alias_id, * table descriptor */ public static void setTaskPlan(String path, String alias, - Operator topOp, MapredWork plan, boolean local, + Operator topOp, MapWork plan, boolean local, TableDesc tt_desc) throws SemanticException { if (path == null || alias == null) { @@ -737,7 +746,7 @@ public static void setTaskPlan(String path, String alias, * @param topOp * current top operator in the path */ - public static void setKeyAndValueDesc(MapredWork plan, + public static void setKeyAndValueDesc(ReduceWork plan, Operator topOp) { if (topOp == null) { return; @@ -778,12 +787,12 @@ public static void setKeyAndValueDescForTaskTree(Task ta } } else if (task instanceof ExecDriver) { MapredWork work = (MapredWork) task.getWork(); - work.deriveExplainAttributes(); + work.getMapWork().deriveExplainAttributes(); HashMap> opMap = work - .getAliasToWork(); + .getMapWork().getAliasToWork(); if (opMap != null && !opMap.isEmpty()) { for (Operator op : opMap.values()) { - setKeyAndValueDesc(work, op); + setKeyAndValueDesc(work.getReduceWork(), op); } } } @@ -804,7 +813,7 @@ public static void setKeyAndValueDescForTaskTree(Task ta */ public static MapredWork getMapRedWork(ParseContext parseCtx) { MapredWork work = getMapRedWorkFromConf(parseCtx.getConf()); - work.setNameToSplitSample(parseCtx.getNameToSplitSample()); + work.getMapWork().setNameToSplitSample(parseCtx.getNameToSplitSample()); return work; } @@ -815,7 +824,8 @@ public static MapredWork getMapRedWork(ParseContext parseCtx) { * @return the new plan */ public static MapredWork getMapRedWorkFromConf(HiveConf conf) { - MapredWork work = new MapredWork(); + MapredWork mrWork = new MapredWork(); + MapWork work = mrWork.getMapWork(); boolean mapperCannotSpanPartns = conf.getBoolVar( @@ -824,11 +834,9 @@ public static MapredWork getMapRedWorkFromConf(HiveConf conf) { 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; + return mrWork; } /** @@ -939,7 +947,7 @@ public static void splitTasks(Operator op, streamDesc = "$INTNAME"; origStreamDesc = streamDesc; int pos = 0; - while (cplan.getAliasToWork().get(streamDesc) != null) { + while (cplan.getMapWork().getAliasToWork().get(streamDesc) != null) { streamDesc = origStreamDesc.concat(String.valueOf(++pos)); } } @@ -947,12 +955,12 @@ public static void splitTasks(Operator op, // TODO: Allocate work to remove the temporary files and make that // dependent on the redTask if (reducer.getClass() == JoinOperator.class) { - cplan.setNeedsTagging(true); + cplan.getReduceWork().setNeedsTagging(true); } } // Add the path to alias mapping - setTaskPlan(taskTmpDir, streamDesc, ts_op, cplan, local, tt_desc); + setTaskPlan(taskTmpDir, streamDesc, ts_op, cplan.getMapWork(), local, tt_desc); opProcCtx.setCurrTopOp(null); opProcCtx.setCurrAliasId(null); opProcCtx.setCurrTask(childTask); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java index c51babb..a1f0bf6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; @@ -77,7 +78,7 @@ public static int getPositionParent(AbstractMapJoinOperator currMapJoinOp) { if (currMapJoinOp != null) { Map>> aliasBucketFileNameMapping = @@ -174,8 +175,8 @@ private static void initMapJoinPlan(AbstractMapJoinOperator oldTask, GenMRProcContext opProcCtx, int pos) throws SemanticException { - MapredWork plan = (MapredWork) oldTask.getWork(); + MapWork plan = ((MapredWork) oldTask.getWork()).getMapWork(); Operator currTopOp = opProcCtx.getCurrTopOp(); List> seenOps = opProcCtx.getSeenOps(); @@ -255,7 +256,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // branches for this mapjoin have been encounered. Join the plan with the plan created // the first time. if (opMapTask == null) { - assert currPlan.getReducer() == null; + assert currPlan.getReduceWork() == null; initMapJoinPlan(mapJoin, ctx, pos); } else { // The current plan can be thrown away after being merged with the diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java index d83fb66..b5614e2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java @@ -134,7 +134,7 @@ private static String genMapJoinLocalWork(MapredWork newWork, MapJoinOperator ma new LinkedHashMap()); for (Map.Entry> entry : - newWork.getAliasToWork().entrySet()) { + newWork.getMapWork().getAliasToWork().entrySet()) { String alias = entry.getKey(); Operator op = entry.getValue(); @@ -162,7 +162,7 @@ private static String genMapJoinLocalWork(MapredWork newWork, MapJoinOperator ma smallTableAliasList.add(alias); // get input path and remove this alias from pathToAlias // because this file will be fetched by fetch operator - LinkedHashMap> pathToAliases = newWork.getPathToAliases(); + LinkedHashMap> pathToAliases = newWork.getMapWork().getPathToAliases(); // keep record all the input path for this alias HashSet pathSet = new HashSet(); @@ -193,7 +193,7 @@ private static String genMapJoinLocalWork(MapredWork newWork, MapJoinOperator ma List partDesc = new ArrayList(); for (String tablePath : pathSet) { - PartitionDesc partitionDesc = newWork.getPathToPartitionInfo().get(tablePath); + PartitionDesc partitionDesc = newWork.getMapWork().getPathToPartitionInfo().get(tablePath); // create fetchwork for non partitioned table if (partitionDesc.getPartSpec() == null || partitionDesc.getPartSpec().size() == 0) { fetchWork = new FetchWork(tablePath, partitionDesc.getTableDesc()); @@ -205,7 +205,7 @@ private static String genMapJoinLocalWork(MapredWork newWork, MapJoinOperator ma } // create fetchwork for partitioned table if (fetchWork == null) { - TableDesc table = newWork.getAliasToPartnInfo().get(alias).getTableDesc(); + TableDesc table = newWork.getMapWork().getAliasToPartnInfo().get(alias).getTableDesc(); fetchWork = new FetchWork(partDir, partDesc, table); } // set alias to fetch work @@ -213,13 +213,13 @@ private static String genMapJoinLocalWork(MapredWork newWork, MapJoinOperator ma } // remove small table ailias from aliasToWork;Avoid concurrent modification for (String alias : smallTableAliasList) { - newWork.getAliasToWork().remove(alias); + newWork.getMapWork().getAliasToWork().remove(alias); } // set up local work - newWork.setMapLocalWork(newLocalWork); + newWork.getMapWork().setMapLocalWork(newLocalWork); // remove reducer - newWork.setReducer(null); + newWork.setReduceWork(null); // return the big table alias if (bigTableAlias == null) { throw new SemanticException("Big Table Alias is null"); @@ -240,8 +240,8 @@ private static String genMapJoinLocalWork(MapredWork newWork, MapJoinOperator ma public static String genMapJoinOpAndLocalWork(MapredWork newWork, JoinOperator op, int mapJoinPos) throws SemanticException { LinkedHashMap, OpParseContext> opParseCtxMap = - newWork.getOpParseCtxMap(); - QBJoinTree newJoinTree = newWork.getJoinTree(); + newWork.getMapWork().getOpParseCtxMap(); + QBJoinTree newJoinTree = newWork.getMapWork().getJoinTree(); // generate the map join operator; already checked the map join MapJoinOperator newMapJoinOp = MapJoinProcessor.convertMapJoin(opParseCtxMap, op, newJoinTree, mapJoinPos, true, false); @@ -256,8 +256,8 @@ public static String genLocalWorkForMapJoin(MapredWork newWork, MapJoinOperator String bigTableAlias = MapJoinProcessor .genMapJoinLocalWork(newWork, newMapJoinOp, mapJoinPos); // clean up the mapred work - newWork.setOpParseCtxMap(null); - newWork.setJoinTree(null); + newWork.getMapWork().setOpParseCtxMap(null); + newWork.getMapWork().setJoinTree(null); return bigTableAlias; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java index c876ab7..33ef581 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.lib.TaskGraphWalker.TaskGraphWalkerContext; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; /** * Common iteration methods for converting joins and sort-merge joins. @@ -119,7 +120,7 @@ protected void replaceTask( } } - public long getTotalKnownInputSize(Context context, MapredWork currWork, + public long getTotalKnownInputSize(Context context, MapWork currWork, Map> pathToAliases, HashMap aliasToSize) throws SemanticException { try { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java index 35dfdc5..87fba2d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java @@ -86,13 +86,13 @@ private void inferBucketingSorting(List mapRedTasks) throws Semantic continue; } - Operator reducer = mapRedTask.getWork().getReducer(); - if (reducer == null) { + if (mapRedTask.getWork().getReduceWork() == null) { continue; } + Operator reducer = mapRedTask.getWork().getReduceWork().getReducer(); // uses sampling, which means it's not bucketed - boolean disableBucketing = mapRedTask.getWork().getSamplingType() > 0; + boolean disableBucketing = mapRedTask.getWork().getMapWork().getSamplingType() > 0; BucketingSortingCtx bCtx = new BucketingSortingCtx(disableBucketing); // RuleRegExp rules are used to match operators anywhere in the tree @@ -145,8 +145,8 @@ private void inferBucketingSorting(List mapRedTasks) throws Semantic topNodes.add(reducer); ogw.startWalking(topNodes, null); - mapRedTask.getWork().getBucketedColsByDirectory().putAll(bCtx.getBucketedColsByDirectory()); - mapRedTask.getWork().getSortedColsByDirectory().putAll(bCtx.getSortedColsByDirectory()); + mapRedTask.getWork().getMapWork().getBucketedColsByDirectory().putAll(bCtx.getBucketedColsByDirectory()); + mapRedTask.getWork().getMapWork().getSortedColsByDirectory().putAll(bCtx.getSortedColsByDirectory()); } } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java index 6d78e33..b48bb1b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java @@ -52,6 +52,8 @@ import org.apache.hadoop.hive.ql.plan.JoinDesc; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.MapWork; +import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PartitionDesc; @@ -108,7 +110,7 @@ public CommonJoinTaskDispatcher(PhysicalContext context) { } // Get the position of the big table for this join operator and the given alias - private int getPosition(MapredWork work, Operator joinOp, + private int getPosition(MapWork work, Operator joinOp, String alias) { Operator parentOp = work.getAliasToWork().get(alias); @@ -127,9 +129,9 @@ private int getPosition(MapredWork work, Operator joinOp */ private void mergeMapJoinTaskWithChildMapJoinTask(MapRedTask task, Configuration conf) { MapRedTask childTask = (MapRedTask) task.getChildTasks().get(0); - MapredWork work = task.getWork(); + MapWork work = task.getWork().getMapWork(); MapredLocalWork localWork = work.getMapLocalWork(); - MapredWork childWork = childTask.getWork(); + MapWork childWork = childTask.getWork().getMapWork(); MapredLocalWork childLocalWork = childWork.getMapLocalWork(); // Can this be merged @@ -256,19 +258,26 @@ private void mergeMapJoinTaskWithChildMapJoinTask(MapRedTask task, Configuration * @param childTask */ private void copyReducerConf(MapRedTask task, MapRedTask childTask) { - MapredWork childWork = childTask.getWork(); + MapredWork mrChildWork = childTask.getWork(); + ReduceWork childWork = childTask.getWork().getReduceWork(); + if (childWork == null) { + return; + } + Operator childReducer = childWork.getReducer(); MapredWork work = task.getWork(); if (childReducer == null) { return; } - work.setReducer(childReducer); - work.setNumReduceTasks(childWork.getNumReduceTasks()); - work.setJoinTree(childWork.getJoinTree()); - work.setNeedsTagging(childWork.getNeedsTagging()); + ReduceWork rWork = new ReduceWork(); + work.setReduceWork(rWork); + rWork.setReducer(childReducer); + rWork.setNumReduceTasks(childWork.getNumReduceTasks()); + work.getMapWork().setJoinTree(mrChildWork.getMapWork().getJoinTree()); + rWork.setNeedsTagging(childWork.getNeedsTagging()); // Make sure the key configuration is correct, clear and regenerate. - work.getTagToValueDesc().clear(); + rWork.getTagToValueDesc().clear(); GenMapRedUtils.setKeyAndValueDescForTaskTree(task); } @@ -303,10 +312,9 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura return; } MapRedTask childTask = (MapRedTask) firstChildTask; - MapredWork mapJoinWork = mapJoinTask.getWork(); + MapWork mapJoinWork = mapJoinTask.getWork().getMapWork(); MapredWork childWork = childTask.getWork(); - Operator childReducer = childWork.getReducer(); - if (childReducer == null) { + if (childWork.getReduceWork() == null) { // Not a MR job, nothing to merge. return; } @@ -316,7 +324,7 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura if (aliasToWork.size() > 1) { return; } - Map> childPathToAliases = childWork.getPathToAliases(); + Map> childPathToAliases = childWork.getMapWork().getPathToAliases(); if (childPathToAliases.size() > 1) { return; } @@ -347,7 +355,7 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura } MapredLocalWork mapJoinLocalWork = mapJoinWork.getMapLocalWork(); - MapredLocalWork childLocalWork = childWork.getMapLocalWork(); + MapredLocalWork childLocalWork = childWork.getMapWork().getMapLocalWork(); // Either of them should not be bucketed if ((mapJoinLocalWork != null && mapJoinLocalWork.getBucketMapjoinContext() != null) || @@ -355,12 +363,12 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura return; } - if (childWork.getAliasToWork().size() > 1) { + if (childWork.getMapWork().getAliasToWork().size() > 1) { return; } Operator childAliasOp = - childWork.getAliasToWork().values().iterator().next(); + childWork.getMapWork().getAliasToWork().values().iterator().next(); if (mapJoinTaskFileSinkOperator.getParentOperators().size() > 1) { return; } @@ -387,10 +395,10 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura parentOps.add(parentFOp); childAliasOp.setParentOperators(parentOps); - mapJoinWork.getAliasToPartnInfo().putAll(childWork.getAliasToPartnInfo()); - for (Map.Entry childWorkEntry : childWork.getPathToPartitionInfo() + mapJoinWork.getAliasToPartnInfo().putAll(childWork.getMapWork().getAliasToPartnInfo()); + for (Map.Entry childWorkEntry : childWork.getMapWork().getPathToPartitionInfo() .entrySet()) { - if (childWork.getAliasToPartnInfo().containsValue(childWorkEntry.getKey())) { + if (childWork.getMapWork().getAliasToPartnInfo().containsValue(childWorkEntry.getKey())) { mapJoinWork.getPathToPartitionInfo() .put(childWorkEntry.getKey(), childWorkEntry.getValue()); } @@ -428,7 +436,7 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura } currTask.setTaskTag(Task.COMMON_JOIN); - MapredWork currWork = currTask.getWork(); + MapWork currWork = currTask.getWork().getMapWork(); // create conditional work list and task list List listWorks = new ArrayList(); @@ -519,7 +527,7 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura if (convertJoinMapJoin) { // create map join task and set big table as bigTablePosition - MapRedTask newTask = convertTaskToMapJoinTask(currWork, bigTablePosition).getFirst(); + MapRedTask newTask = convertTaskToMapJoinTask(currTask.getWork(), bigTablePosition).getFirst(); newTask.setTaskTag(Task.MAPJOIN_ONLY_NOBACKUP); replaceTask(currTask, newTask, physicalContext); @@ -555,7 +563,7 @@ private void mergeMapJoinTaskWithMapReduceTask(MapRedTask mapJoinTask, Configura } // deep copy a new mapred work from xml // Once HIVE-4396 is in, it would be faster to use a cheaper method to clone the plan - String xml = currWork.toXML(); + String xml = currTask.getWork().toXML(); InputStream in = new ByteArrayInputStream(xml.getBytes("UTF-8")); MapredWork newWork = Utilities.deserializeMapRedWork(in, physicalContext.getConf()); @@ -642,14 +650,15 @@ private boolean checkOperatorOKMapJoinConversion(Operator reducerOp = work.getReducer(); + Operator reducerOp = rWork.getReducer(); if (reducerOp instanceof JoinOperator) { /* Is any operator present, which prevents the conversion */ - Map> aliasToWork = work.getAliasToWork(); + Map> aliasToWork = mWork.getAliasToWork(); for (Operator op : aliasToWork.values()) { if (!checkOperatorOKMapJoinConversion(op)) { return null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java index 2bbb278..fbe56d4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hive.ql.optimizer.physical; import java.io.ByteArrayInputStream; -import java.io.File; import java.io.Serializable; import java.io.UnsupportedEncodingException; import java.util.ArrayList; @@ -50,6 +49,7 @@ import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.JoinDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; @@ -151,7 +151,7 @@ public static void processSkewJoin(JoinOperator joinOp, List> listTasks = new ArrayList>(); MapredWork currPlan = (MapredWork) currTask.getWork(); - TableDesc keyTblDesc = (TableDesc) currPlan.getKeyDesc().clone(); + TableDesc keyTblDesc = (TableDesc) currPlan.getReduceWork().getKeyDesc().clone(); List joinKeys = Utilities .getColumnNames(keyTblDesc.getProperties()); List joinKeyTypes = Utilities.getColumnTypes(keyTblDesc @@ -232,7 +232,7 @@ public static void processSkewJoin(JoinOperator joinOp, for (int i = 0; i < numAliases - 1; i++) { Byte src = tags[i]; - MapredWork newPlan = PlanUtils.getMapRedWork(); + MapWork newPlan = PlanUtils.getMapRedWork().getMapWork(); // This code has been only added for testing boolean mapperCannotSpanPartns = @@ -276,7 +276,7 @@ public static void processSkewJoin(JoinOperator joinOp, newPlan.getPathToPartitionInfo().put(bigKeyDirPath, part); newPlan.getAliasToPartnInfo().put(alias, part); - Operator reducer = clonePlan.getReducer(); + Operator reducer = clonePlan.getReduceWork().getReducer(); assert reducer instanceof JoinOperator; JoinOperator cloneJoinOp = (JoinOperator) reducer; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java index efaeccf..010ac54 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java @@ -48,8 +48,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.ConditionalResolver; import org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin; -import - org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin.ConditionalResolverCommonJoinCtx; +import org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin.ConditionalResolverCommonJoinCtx; import org.apache.hadoop.hive.ql.plan.ConditionalResolverSkewJoin; import org.apache.hadoop.hive.ql.plan.ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx; import org.apache.hadoop.hive.ql.plan.ConditionalWork; @@ -98,14 +97,14 @@ private void processCurrentTask(Task currTask, ConditionalTask conditionalTask) throws SemanticException { // get current mapred work and its local work MapredWork mapredWork = (MapredWork) currTask.getWork(); - MapredLocalWork localwork = mapredWork.getMapLocalWork(); + MapredLocalWork localwork = mapredWork.getMapWork().getMapLocalWork(); if (localwork != null) { // get the context info and set up the shared tmp URI Context ctx = physicalContext.getContext(); String tmpFileURI = Utilities.generateTmpURI(ctx.getLocalTmpFileURI(), currTask.getId()); localwork.setTmpFileURI(tmpFileURI); String hdfsTmpURI = Utilities.generateTmpURI(ctx.getMRTmpFileURI(), currTask.getId()); - mapredWork.setTmpHDFSFileURI(hdfsTmpURI); + mapredWork.getMapWork().setTmpHDFSFileURI(hdfsTmpURI); // create a task for this local work; right now, this local work is shared // by the original MapredTask and this new generated MapredLocalTask. MapredLocalTask localTask = (MapredLocalTask) TaskFactory.get(localwork, physicalContext @@ -134,7 +133,7 @@ private void processCurrentTask(Task currTask, newLocalWork.setTmpFileURI(tmpFileURI); newLocalWork.setInputFileChangeSensitive(localwork.getInputFileChangeSensitive()); newLocalWork.setBucketMapjoinContext(localwork.copyPartSpecMappingOnly()); - mapredWork.setMapLocalWork(newLocalWork); + mapredWork.getMapWork().setMapLocalWork(newLocalWork); // get all parent tasks List> parentTasks = currTask.getParentTasks(); currTask.setParentTasks(null); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java index 15653bf..b208d64 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hive.ql.lib.RuleRegExp; import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PartitionDesc; @@ -188,7 +189,7 @@ public MetadataOnlyTaskDispatcher(PhysicalContext context) { physicalContext = context; } - private String getAliasForTableScanOperator(MapredWork work, + private String getAliasForTableScanOperator(MapWork work, TableScanOperator tso) { for (Map.Entry> entry : @@ -211,7 +212,7 @@ private PartitionDesc changePartitionToMetadataOnly(PartitionDesc desc) { return desc; } - private List getPathsForAlias(MapredWork work, String alias) { + private List getPathsForAlias(MapWork work, String alias) { List paths = new ArrayList(); for (Map.Entry> entry : work.getPathToAliases().entrySet()) { @@ -223,7 +224,7 @@ private PartitionDesc changePartitionToMetadataOnly(PartitionDesc desc) { return paths; } - private void processAlias(MapredWork work, String alias) { + private void processAlias(MapWork work, String alias) { // Change the alias partition desc PartitionDesc aliasPartn = work.getAliasToPartnInfo().get(alias); changePartitionToMetadataOnly(aliasPartn); @@ -247,7 +248,7 @@ private String encode(Map partSpec) { return partSpec.toString().replaceAll("[:/#\\?]", "_"); } - private void convertToMetadataOnlyQuery(MapredWork work, + private void convertToMetadataOnlyQuery(MapWork work, TableScanOperator tso) { String alias = getAliasForTableScanOperator(work, tso); processAlias(work, alias); @@ -306,7 +307,7 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) while (iterator.hasNext()) { TableScanOperator tso = iterator.next(); LOG.info("Metadata only table scan for " + tso.getConf().getAlias()); - convertToMetadataOnlyQuery((MapredWork) task.getWork(), tso); + convertToMetadataOnlyQuery(((MapredWork) task.getWork()).getMapWork(), tso); } return null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SamplingOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SamplingOptimizer.java index c0c232e..2e1d15c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SamplingOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SamplingOptimizer.java @@ -27,7 +27,9 @@ import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.ReduceWork; /** * Mark final MapredWork for ORDER BY to use sampling and set number of reduce task as -1 @@ -39,12 +41,16 @@ public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { if (!(task instanceof MapRedTask) || !((MapRedTask)task).getWork().isFinalMapRed()) { continue; // this could be replaced by bucketing on RS + bucketed fetcher for next MR } - MapredWork mapreWork = ((MapRedTask) task).getWork(); - if (mapreWork.getNumReduceTasks() != 1 || mapreWork.getAliasToWork().size() != 1 || - mapreWork.getSamplingType() > 0 || mapreWork.getReducer() == null) { + MapredWork mrWork = ((MapRedTask) task).getWork(); + MapWork mapWork = mrWork.getMapWork(); + ReduceWork reduceWork = mrWork.getReduceWork(); + + if (reduceWork == null || reduceWork.getNumReduceTasks() != 1 + || mapWork.getAliasToWork().size() != 1 || mapWork.getSamplingType() > 0 + || reduceWork.getReducer() == null) { continue; } - Operator operator = mapreWork.getAliasToWork().values().iterator().next(); + Operator operator = mapWork.getAliasToWork().values().iterator().next(); if (!(operator instanceof TableScanOperator)) { continue; } @@ -55,8 +61,8 @@ public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { continue; } child.getConf().setNumReducers(-1); - mapreWork.setNumReduceTasks(-1); - mapreWork.setSamplingType(MapredWork.SAMPLING_ON_START); + reduceWork.setNumReduceTasks(-1); + mapWork.setSamplingType(MapWork.SAMPLING_ON_START); } return pctx; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java index 88786ff..f48d118 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java @@ -74,7 +74,7 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) Task task = (Task) nd; if (!task.isMapRedTask() || task instanceof ConditionalTask - || ((MapredWork) task.getWork()).getReducer() == null) { + || ((MapredWork) task.getWork()).getReduceWork() == null) { return null; } @@ -94,7 +94,9 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) // iterator the reducer operator tree ArrayList topNodes = new ArrayList(); - topNodes.add(((MapredWork) task.getWork()).getReducer()); + if (((MapredWork)task.getWork()).getReduceWork() != null) { + topNodes.add(((MapredWork) task.getWork()).getReduceWork().getReducer()); + } ogw.startWalking(topNodes, null); return null; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java index af56857..0d65b76 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java @@ -52,10 +52,12 @@ import org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin.ConditionalResolverCommonJoinCtx; import org.apache.hadoop.hive.ql.plan.ConditionalWork; import org.apache.hadoop.hive.ql.plan.FetchWork; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; /** @@ -72,7 +74,7 @@ public SortMergeJoinTaskDispatcher(PhysicalContext context) { // Convert the work in the SMB plan to a regular join // Note that the operator tree is not fixed, only the path/alias mappings in the // plan are fixed. The operator tree will still contain the SMBJoinOperator - private void genSMBJoinWork(MapredWork currWork, SMBMapJoinOperator smbJoinOp) { + private void genSMBJoinWork(MapWork currWork, SMBMapJoinOperator smbJoinOp) { // Remove the paths which are not part of aliasToPartitionInfo Map aliasToPartitionInfo = currWork.getAliasToPartnInfo(); List removePaths = new ArrayList(); @@ -158,7 +160,7 @@ private MapredWork convertSMBWorkToJoinWork(MapredWork currWork, SMBMapJoinOpera physicalContext.getParseContext().getOpParseCtx(); opParseContextMap.put(newSMBJoinOp, opParseContextMap.get(oldSMBJoinOp)); // change the newly created map-red plan as if it was a join operator - genSMBJoinWork(currJoinWork, newSMBJoinOp); + genSMBJoinWork(currJoinWork.getMapWork(), newSMBJoinOp); return currJoinWork; } catch (Exception e) { e.printStackTrace(); @@ -181,17 +183,18 @@ private MapredWork convertSMBWorkToJoinWork(MapredWork currWork, SMBMapJoinOpera // generate the map join operator; already checked the map join MapJoinOperator newMapJoinOp = getMapJoinOperator(newTask, newWork, smbJoinOp, joinTree, bigTablePosition); + // The reducer needs to be restored - Consider a query like: // select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; // The reducer contains a groupby, which needs to be restored. - Operator reducer = newWork.getReducer(); + ReduceWork rWork = newWork.getReduceWork(); // create the local work for this plan String bigTableAlias = MapJoinProcessor.genLocalWorkForMapJoin(newWork, newMapJoinOp, bigTablePosition); // restore the reducer - newWork.setReducer(reducer); + newWork.setReduceWork(rWork); return new ObjectPair(newTask, bigTableAlias); } @@ -259,10 +262,10 @@ private boolean isEligibleForOptimization(SMBMapJoinOperator originalSMBJoinOp) MapredWork currJoinWork = convertSMBWorkToJoinWork(currWork, originalSMBJoinOp); SMBMapJoinOperator newSMBJoinOp = getSMBMapJoinOp(currJoinWork); - currWork.setOpParseCtxMap(parseCtx.getOpParseCtx()); - currWork.setJoinTree(joinTree); - currJoinWork.setOpParseCtxMap(parseCtx.getOpParseCtx()); - currJoinWork.setJoinTree(joinTree); + currWork.getMapWork().setOpParseCtxMap(parseCtx.getOpParseCtx()); + currWork.getMapWork().setJoinTree(joinTree); + currJoinWork.getMapWork().setOpParseCtxMap(parseCtx.getOpParseCtx()); + currJoinWork.getMapWork().setJoinTree(joinTree); // create conditional work list and task list List listWorks = new ArrayList(); @@ -272,7 +275,7 @@ private boolean isEligibleForOptimization(SMBMapJoinOperator originalSMBJoinOp) HashMap> aliasToTask = new HashMap>(); // Note that pathToAlias will behave as if the original plan was a join plan - HashMap> pathToAliases = currJoinWork.getPathToAliases(); + HashMap> pathToAliases = currJoinWork.getMapWork().getPathToAliases(); // generate a map join task for the big table SMBJoinDesc originalSMBJoinDesc = originalSMBJoinOp.getConf(); @@ -289,7 +292,7 @@ private boolean isEligibleForOptimization(SMBMapJoinOperator originalSMBJoinOp) HashMap aliasToSize = new HashMap(); Configuration conf = context.getConf(); try { - long aliasTotalKnownInputSize = getTotalKnownInputSize(context, currJoinWork, + long aliasTotalKnownInputSize = getTotalKnownInputSize(context, currJoinWork.getMapWork(), pathToAliases, aliasToSize); String xml = currJoinWork.toXML(); @@ -339,8 +342,8 @@ private boolean isEligibleForOptimization(SMBMapJoinOperator originalSMBJoinOp) listWorks.add(currTask.getWork()); listTasks.add(currTask); // clear JoinTree and OP Parse Context - currWork.setOpParseCtxMap(null); - currWork.setJoinTree(null); + currWork.getMapWork().setOpParseCtxMap(null); + currWork.getMapWork().setJoinTree(null); // create conditional task and insert conditional task into task tree ConditionalWork cndWork = new ConditionalWork(listWorks); @@ -417,9 +420,9 @@ private SMBMapJoinOperator getSMBMapJoinOp(Operator curr } private SMBMapJoinOperator getSMBMapJoinOp(MapredWork work) throws SemanticException { - if (work != null) { - Operator reducer = work.getReducer(); - for (Operator op : work.getAliasToWork().values()) { + if (work != null && work.getReduceWork() != null) { + Operator reducer = work.getReduceWork().getReducer(); + for (Operator op : work.getMapWork().getAliasToWork().values()) { SMBMapJoinOperator smbMapJoinOp = getSMBMapJoinOp(op, reducer); if (smbMapJoinOp != null) { return smbMapJoinOp; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java index 8bce7c3..5c6751c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; @@ -162,7 +163,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, HiveIndexQueryContext queryContext = queryContexts.get(chosenIndex); // prepare the map reduce job to use indexing - MapredWork work = currentTask.getWork(); + MapWork work = currentTask.getWork().getMapWork(); work.setInputformat(queryContext.getIndexInputFormat()); work.addIndexIntermediateFile(queryContext.getIndexIntermediateFile()); // modify inputs based on index query @@ -204,7 +205,7 @@ private void rewriteForIndexes(ExprNodeDesc predicate, List indexes, // check the size try { - ContentSummary inputSummary = Utilities.getInputSummary(pctx.getContext(), task.getWork(), null); + ContentSummary inputSummary = Utilities.getInputSummary(pctx.getContext(), task.getWork().getMapWork(), null); long inputSize = inputSummary.getLength(); if (!indexHandler.checkQuerySize(inputSize, pctx.getConf())) { queryContext.setQueryTasks(null); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java index 61c1be7..ef86266 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java @@ -93,7 +93,7 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) GraphWalker ogw = new DefaultGraphWalker(dispatcher); ArrayList topNodes = new ArrayList(); if (task.getWork() instanceof MapredWork) { - topNodes.addAll(((MapredWork)task.getWork()).getAliasToWork().values()); + topNodes.addAll(((MapredWork)task.getWork()).getMapWork().getAliasToWork().values()); } else { return null; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java index fb2b537..374b138 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java @@ -84,6 +84,7 @@ import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.LoadFileDesc; import org.apache.hadoop.hive.ql.plan.LoadTableDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; @@ -350,7 +351,7 @@ public void compile(final ParseContext pCtx, final List op) { + private void setInputFormat(MapWork work, Operator op) { if (op.isUseBucketizedHiveInputFormat()) { work.setUseBucketizedHiveInputFormat(true); return; @@ -366,7 +367,7 @@ private void setInputFormat(MapredWork work, Operator op // loop over all the tasks recursively private void setInputFormat(Task task) { if (task instanceof ExecDriver) { - MapredWork work = (MapredWork) task.getWork(); + MapWork work = ((MapredWork) task.getWork()).getMapWork(); HashMap> opMap = work.getAliasToWork(); if (!opMap.isEmpty()) { for (Operator op : opMap.values()) { @@ -392,16 +393,16 @@ private void setInputFormat(Task task) { private void generateCountersTask(Task task) { if (task instanceof ExecDriver) { HashMap> opMap = ((MapredWork) task - .getWork()).getAliasToWork(); + .getWork()).getMapWork().getAliasToWork(); if (!opMap.isEmpty()) { for (Operator op : opMap.values()) { generateCountersOperator(op); } } - Operator reducer = ((MapredWork) task.getWork()) - .getReducer(); - if (reducer != null) { + if (((MapredWork)task.getWork()).getReduceWork() != null) { + Operator reducer = ((MapredWork) task.getWork()).getReduceWork() + .getReducer(); LOG.info("Generating counters for operator " + reducer); generateCountersOperator(reducer); } @@ -457,7 +458,7 @@ private void breakTaskTree(Task task) { if (task instanceof ExecDriver) { HashMap> opMap = ((MapredWork) task - .getWork()).getAliasToWork(); + .getWork()).getMapWork().getAliasToWork(); if (!opMap.isEmpty()) { for (Operator op : opMap.values()) { breakOperatorTree(op); @@ -560,12 +561,12 @@ private void getLeafTasks(Task task, * Make a best guess at trying to find the number of reducers */ private static int getNumberOfReducers(MapredWork mrwork, HiveConf conf) { - if (mrwork.getReducer() == null) { + if (mrwork.getReduceWork() == null) { return 0; } - if (mrwork.getNumReduceTasks() >= 0) { - return mrwork.getNumReduceTasks(); + if (mrwork.getReduceWork().getNumReduceTasks() >= 0) { + return mrwork.getReduceWork().getNumReduceTasks(); } return conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS); @@ -600,7 +601,7 @@ public boolean accept(Path file) { for (ExecDriver mrtask : mrtasks) { try { ContentSummary inputSummary = Utilities.getInputSummary - (ctx, (MapredWork) mrtask.getWork(), p); + (ctx, ((MapredWork) mrtask.getWork()).getMapWork(), p); int numReducers = getNumberOfReducers(mrtask.getWork(), conf); long estimatedInput; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java new file mode 100644 index 0000000..20d4809 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol; +import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.SortCol; + +@SuppressWarnings({"serial", "deprecation"}) +public abstract class BaseWork extends AbstractOperatorDesc { + + private boolean gatheringStats; + + // If this map reduce task has a FileSinkOperator, and bucketing/sorting metadata can be + // inferred about the data being written by that operator, these are mappings from the directory + // that operator writes into to the bucket/sort columns for that data. + private final Map> bucketedColsByDirectory = + new HashMap>(); + private final Map> sortedColsByDirectory = + new HashMap>(); + + @Explain(displayName = "Path -> Bucketed Columns", normalExplain = false) + public Map> getBucketedColsByDirectory() { + return bucketedColsByDirectory; + } + + @Explain(displayName = "Path -> Sorted Columns", normalExplain = false) + public Map> getSortedColsByDirectory() { + return sortedColsByDirectory; + } + + public void setGatheringStats(boolean gatherStats) { + this.gatheringStats = gatherStats; + } + + public boolean isGatheringStats() { + return this.gatheringStats; + } + + protected abstract List> getAllRootOperators(); + + public List> getAllOperators() { + + List> returnList = new ArrayList>(); + List> opList = getAllRootOperators(); + + //recursively add all children + while (!opList.isEmpty()) { + Operator op = opList.remove(0); + if (op.getChildOperators() != null) { + opList.addAll(op.getChildOperators()); + } + returnList.add(op); + } + + return returnList; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java index 2ad8f78..89e4569 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java @@ -229,7 +229,7 @@ private void generateActualTasks(HiveConf conf, List ptpi = work.getPathToPartitionInfo(); + Map ptpi = work.getMapWork().getPathToPartitionInfo(); assert ptpi.size() == 1; String path = ptpi.keySet().iterator().next(); PartitionDesc partDesc = ptpi.get(path); @@ -237,7 +237,7 @@ private void generateActualTasks(HiveConf conf, List> pta = work.getPathToAliases(); + Map> pta = work.getMapWork().getPathToAliases(); assert pta.size() == 1; path = pta.keySet().iterator().next(); ArrayList aliases = pta.get(path); @@ -255,7 +255,7 @@ private void generateActualTasks(HiveConf conf, List 0) { + MapWork mWork = work.getMapWork(); + ReduceWork rWork = work.getReduceWork(); + if (rWork != null) { int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); int reducers = (int) ((totalSize + targetSize - 1) / targetSize); reducers = Math.max(1, reducers); reducers = Math.min(maxReducers, reducers); - work.setNumReduceTasks(reducers); + rWork.setNumReduceTasks(reducers); } - work.setMaxSplitSize(targetSize); - work.setMinSplitSize(targetSize); - work.setMinSplitSizePerNode(targetSize); - work.setMinSplitSizePerRack(targetSize); + mWork.setMaxSplitSize(targetSize); + mWork.setMinSplitSize(targetSize); + mWork.setMinSplitSizePerNode(targetSize); + mWork.setMinSplitSizePerRack(targetSize); } private static class AverageSize { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java index d2a52c9..a3408a0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java @@ -32,4 +32,6 @@ boolean normalExplain() default true; boolean displayOnlyOnTrue() default false; + + boolean skipHeader() default false; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java new file mode 100644 index 0000000..f13089c --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java @@ -0,0 +1,449 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorUtils; +import org.apache.hadoop.hive.ql.parse.OpParseContext; +import org.apache.hadoop.hive.ql.parse.QBJoinTree; +import org.apache.hadoop.hive.ql.parse.SplitSample; +import org.apache.hadoop.mapred.JobConf; +import org.apache.log4j.Logger; + + +@SuppressWarnings({"serial", "deprecation"}) +public class MapWork extends BaseWork { + Logger LOG = Logger.getLogger(ReduceWork.class); + + private boolean hadoopSupportsSplittable; + + // map side work + // use LinkedHashMap to make sure the iteration order is + // deterministic, to ease testing + private LinkedHashMap> pathToAliases; + + private LinkedHashMap pathToPartitionInfo; + + private LinkedHashMap> aliasToWork; + + private LinkedHashMap aliasToPartnInfo = new LinkedHashMap(); + + private HashMap nameToSplitSample; + + private MapredLocalWork mapLocalWork; + private String tmpHDFSFileURI; + + private String inputformat; + + private String indexIntermediateFile; + + private Integer numMapTasks; + private Long maxSplitSize; + private Long minSplitSize; + private Long minSplitSizePerNode; + private Long minSplitSizePerRack; + + //use sampled partitioning + private int samplingType; + + public static final int SAMPLING_ON_PREV_MR = 1; // todo HIVE-3841 + public static final int SAMPLING_ON_START = 2; // sampling on task running + + // the following two are used for join processing + private QBJoinTree joinTree; + private LinkedHashMap, OpParseContext> opParseCtxMap; + + private boolean mapperCannotSpanPartns; + + // used to indicate the input is sorted, and so a BinarySearchRecordReader shoudl be used + private boolean inputFormatSorted = false; + + private transient boolean useBucketizedHiveInputFormat; + + public MapWork() { + } + + @Explain(displayName = "Path -> Alias", normalExplain = false) + public LinkedHashMap> getPathToAliases() { + return pathToAliases; + } + + public void setPathToAliases( + final LinkedHashMap> pathToAliases) { + this.pathToAliases = pathToAliases; + } + + /** + * This is used to display and verify output of "Path -> Alias" in test framework. + * + * QTestUtil masks "Path -> Alias" and makes verification impossible. + * By keeping "Path -> Alias" intact and adding a new display name which is not + * masked by QTestUtil by removing prefix. + * + * Notes: we would still be masking for intermediate directories. + * + * @return + */ + @Explain(displayName = "Truncated Path -> Alias", normalExplain = false) + public Map> getTruncatedPathToAliases() { + Map> trunPathToAliases = new LinkedHashMap>(); + Iterator>> itr = this.pathToAliases.entrySet().iterator(); + while (itr.hasNext()) { + final Entry> entry = itr.next(); + String origiKey = entry.getKey(); + String newKey = PlanUtils.removePrefixFromWarehouseConfig(origiKey); + ArrayList value = entry.getValue(); + trunPathToAliases.put(newKey, value); + } + return trunPathToAliases; + } + + @Explain(displayName = "Path -> Partition", normalExplain = false) + public LinkedHashMap getPathToPartitionInfo() { + return pathToPartitionInfo; + } + + public void setPathToPartitionInfo( + final LinkedHashMap pathToPartitionInfo) { + this.pathToPartitionInfo = pathToPartitionInfo; + } + + /** + * Derive additional attributes to be rendered by EXPLAIN. + */ + public void deriveExplainAttributes() { + if (pathToPartitionInfo != null) { + for (Map.Entry entry : pathToPartitionInfo + .entrySet()) { + entry.getValue().deriveBaseFileName(entry.getKey()); + } + } + if (mapLocalWork != null) { + mapLocalWork.deriveExplainAttributes(); + } + } + + /** + * @return the aliasToPartnInfo + */ + public LinkedHashMap getAliasToPartnInfo() { + return aliasToPartnInfo; + } + + /** + * @param aliasToPartnInfo + * the aliasToPartnInfo to set + */ + public void setAliasToPartnInfo( + LinkedHashMap aliasToPartnInfo) { + this.aliasToPartnInfo = aliasToPartnInfo; + } + + @Explain(displayName = "Alias -> Map Operator Tree") + public LinkedHashMap> getAliasToWork() { + return aliasToWork; + } + + public void setAliasToWork( + final LinkedHashMap> aliasToWork) { + this.aliasToWork = aliasToWork; + } + + /** + * @return the mapredLocalWork + */ + @Explain(displayName = "Local Work") + public MapredLocalWork getMapLocalWork() { + return mapLocalWork; + } + + /** + * @param mapLocalWork + * the mapredLocalWork to set + */ + public void setMapLocalWork(final MapredLocalWork mapLocalWork) { + this.mapLocalWork = mapLocalWork; + } + + + @Explain(displayName = "Split Sample") + public HashMap getNameToSplitSample() { + return nameToSplitSample; + } + + public void setNameToSplitSample(HashMap nameToSplitSample) { + this.nameToSplitSample = nameToSplitSample; + } + + public Integer getNumMapTasks() { + return numMapTasks; + } + + public void setNumMapTasks(Integer numMapTasks) { + this.numMapTasks = numMapTasks; + } + + @SuppressWarnings("nls") + public void addMapWork(String path, String alias, Operator work, + PartitionDesc pd) { + ArrayList curAliases = pathToAliases.get(path); + if (curAliases == null) { + assert (pathToPartitionInfo.get(path) == null); + curAliases = new ArrayList(); + pathToAliases.put(path, curAliases); + pathToPartitionInfo.put(path, pd); + } else { + assert (pathToPartitionInfo.get(path) != null); + } + + for (String oneAlias : curAliases) { + if (oneAlias.equals(alias)) { + throw new RuntimeException("Multiple aliases named: " + alias + + " for path: " + path); + } + } + curAliases.add(alias); + + if (aliasToWork.get(alias) != null) { + throw new RuntimeException("Existing work for alias: " + alias); + } + aliasToWork.put(alias, work); + } + + public boolean isInputFormatSorted() { + return inputFormatSorted; + } + + public void setInputFormatSorted(boolean inputFormatSorted) { + this.inputFormatSorted = inputFormatSorted; + } + + public void resolveDynamicPartitionStoredAsSubDirsMerge(HiveConf conf, Path path, + TableDesc tblDesc, ArrayList aliases, PartitionDesc partDesc) { + pathToAliases.put(path.toString(), aliases); + pathToPartitionInfo.put(path.toString(), partDesc); + } + + /** + * For each map side operator - stores the alias the operator is working on + * behalf of in the operator runtime state. This is used by reduce sink + * operator - but could be useful for debugging as well. + */ + private void setAliases() { + if(aliasToWork == null) { + return; + } + for (String oneAlias : aliasToWork.keySet()) { + aliasToWork.get(oneAlias).setAlias(oneAlias); + } + } + + @Override + protected List> getAllRootOperators() { + ArrayList> opList = new ArrayList>(); + + Map> pa = getPathToAliases(); + if (pa != null) { + for (List ls : pa.values()) { + for (String a : ls) { + Operator op = getAliasToWork().get(a); + if (op != null ) { + opList.add(op); + } + } + } + } + return opList; + } + + public void mergeAliasedInput(String alias, String pathDir, PartitionDesc partitionInfo) { + ArrayList aliases = pathToAliases.get(pathDir); + if (aliases == null) { + aliases = new ArrayList(Arrays.asList(alias)); + pathToAliases.put(pathDir, aliases); + pathToPartitionInfo.put(pathDir, partitionInfo); + } else { + aliases.add(alias); + } + } + + public void initialize() { + setAliases(); + } + + public Long getMaxSplitSize() { + return maxSplitSize; + } + + public void setMaxSplitSize(Long maxSplitSize) { + this.maxSplitSize = maxSplitSize; + } + + public Long getMinSplitSize() { + return minSplitSize; + } + + public void setMinSplitSize(Long minSplitSize) { + this.minSplitSize = minSplitSize; + } + + public Long getMinSplitSizePerNode() { + return minSplitSizePerNode; + } + + public void setMinSplitSizePerNode(Long minSplitSizePerNode) { + this.minSplitSizePerNode = minSplitSizePerNode; + } + + public Long getMinSplitSizePerRack() { + return minSplitSizePerRack; + } + + public void setMinSplitSizePerRack(Long minSplitSizePerRack) { + this.minSplitSizePerRack = minSplitSizePerRack; + } + + public String getInputformat() { + return inputformat; + } + + public void setInputformat(String inputformat) { + this.inputformat = inputformat; + } + public boolean isUseBucketizedHiveInputFormat() { + return useBucketizedHiveInputFormat; + } + + public void setUseBucketizedHiveInputFormat(boolean useBucketizedHiveInputFormat) { + this.useBucketizedHiveInputFormat = useBucketizedHiveInputFormat; + } + + public QBJoinTree getJoinTree() { + return joinTree; + } + + public void setJoinTree(QBJoinTree joinTree) { + this.joinTree = joinTree; + } + + public void setMapperCannotSpanPartns(boolean mapperCannotSpanPartns) { + this.mapperCannotSpanPartns = mapperCannotSpanPartns; + } + + public boolean isMapperCannotSpanPartns() { + return this.mapperCannotSpanPartns; + } + + public boolean getHadoopSupportsSplittable() { + return hadoopSupportsSplittable; + } + + public void setHadoopSupportsSplittable(boolean hadoopSupportsSplittable) { + this.hadoopSupportsSplittable = hadoopSupportsSplittable; + } + + public String getIndexIntermediateFile() { + return indexIntermediateFile; + } + + public ArrayList getAliases() { + return new ArrayList(aliasToWork.keySet()); + } + + public ArrayList> getWorks() { + return new ArrayList>(aliasToWork.values()); + } + + public ArrayList getPaths() { + return new ArrayList(pathToAliases.keySet()); + } + + public ArrayList getPartitionDescs() { + return new ArrayList(aliasToPartnInfo.values()); + } + + public + LinkedHashMap, OpParseContext> getOpParseCtxMap() { + return opParseCtxMap; + } + + public void setOpParseCtxMap( + LinkedHashMap, OpParseContext> opParseCtxMap) { + this.opParseCtxMap = opParseCtxMap; + } + + public String getTmpHDFSFileURI() { + return tmpHDFSFileURI; + } + + public void setTmpHDFSFileURI(String tmpHDFSFileURI) { + this.tmpHDFSFileURI = tmpHDFSFileURI; + } + + public void mergingInto(MapWork mapWork) { + // currently, this is sole field affecting mergee task + mapWork.useBucketizedHiveInputFormat |= useBucketizedHiveInputFormat; + } + + public void addIndexIntermediateFile(String fileName) { + if (this.indexIntermediateFile == null) { + this.indexIntermediateFile = fileName; + } else { + this.indexIntermediateFile += "," + fileName; + } + } + + public int getSamplingType() { + return samplingType; + } + + public void setSamplingType(int samplingType) { + this.samplingType = samplingType; + } + + @Explain(displayName = "Sampling") + public String getSamplingTypeString() { + return samplingType == 1 ? "SAMPLING_ON_PREV_MR" : + samplingType == 2 ? "SAMPLING_ON_START" : null; + } + + public void configureJobConf(JobConf job) { + for (PartitionDesc partition : aliasToPartnInfo.values()) { + PlanUtils.configureJobConf(partition.getTableDesc(), job); + } + Collection> mappers = aliasToWork.values(); + for (FileSinkOperator fs : OperatorUtils.findOperators(mappers, FileSinkOperator.class)) { + PlanUtils.configureJobConf(fs.getConf().getTableInfo(), job); + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java index 7cbb1ff..1c14418 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java @@ -20,28 +20,13 @@ import java.io.ByteArrayOutputStream; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol; -import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.SortCol; -import org.apache.hadoop.hive.ql.parse.OpParseContext; -import org.apache.hadoop.hive.ql.parse.QBJoinTree; -import org.apache.hadoop.hive.ql.parse.SplitSample; import org.apache.hadoop.mapred.JobConf; + /** * MapredWork. * @@ -49,342 +34,53 @@ @Explain(displayName = "Map Reduce") public class MapredWork extends AbstractOperatorDesc { private static final long serialVersionUID = 1L; - private String command; - // map side work - // use LinkedHashMap to make sure the iteration order is - // deterministic, to ease testing - private LinkedHashMap> pathToAliases; - - private LinkedHashMap pathToPartitionInfo; - - private LinkedHashMap> aliasToWork; - - private LinkedHashMap aliasToPartnInfo; - - private HashMap nameToSplitSample; - - // map<->reduce interface - // schema of the map-reduce 'key' object - this is homogeneous - private TableDesc keyDesc; - - // schema of the map-reduce 'val' object - this is heterogeneous - private List tagToValueDesc; - - private Operator reducer; - - private Integer numReduceTasks; - private Integer numMapTasks; - private Long maxSplitSize; - private Long minSplitSize; - private Long minSplitSizePerNode; - private Long minSplitSizePerRack; - - private boolean needsTagging; - private boolean hadoopSupportsSplittable; - - private MapredLocalWork mapLocalWork; - private String inputformat; - private String indexIntermediateFile; - private boolean gatheringStats; - - private String tmpHDFSFileURI; - - private LinkedHashMap, OpParseContext> opParseCtxMap; - private QBJoinTree joinTree; + private MapWork mapWork = new MapWork(); + private ReduceWork reduceWork = null; - private boolean mapperCannotSpanPartns; - - // used to indicate the input is sorted, and so a BinarySearchRecordReader shoudl be used - private boolean inputFormatSorted = false; - - private transient boolean useBucketizedHiveInputFormat; - - // if this is true, this means that this is the map reduce task which writes the final data, - // ignoring the optional merge task - private boolean finalMapRed = false; - - // If this map reduce task has a FileSinkOperator, and bucketing/sorting metadata can be - // inferred about the data being written by that operator, these are mappings from the directory - // that operator writes into to the bucket/sort columns for that data. - private final Map> bucketedColsByDirectory = - new HashMap>(); - private final Map> sortedColsByDirectory = - new HashMap>(); - - // use sampled partitioning - private int samplingType; - - public static final int SAMPLING_ON_PREV_MR = 1; // todo HIVE-3841 - public static final int SAMPLING_ON_START = 2; // sampling on task running - - public MapredWork() { - aliasToPartnInfo = new LinkedHashMap(); - } - - public MapredWork( - final String command, - final LinkedHashMap> pathToAliases, - final LinkedHashMap pathToPartitionInfo, - final LinkedHashMap> aliasToWork, - final TableDesc keyDesc, List tagToValueDesc, - final Operator reducer, final Integer numReduceTasks, - final MapredLocalWork mapLocalWork, - final boolean hadoopSupportsSplittable) { - this.command = command; - this.pathToAliases = pathToAliases; - this.pathToPartitionInfo = pathToPartitionInfo; - this.aliasToWork = aliasToWork; - this.keyDesc = keyDesc; - this.tagToValueDesc = tagToValueDesc; - this.reducer = reducer; - this.numReduceTasks = numReduceTasks; - this.mapLocalWork = mapLocalWork; - aliasToPartnInfo = new LinkedHashMap(); - this.hadoopSupportsSplittable = hadoopSupportsSplittable; - maxSplitSize = null; - minSplitSize = null; - minSplitSizePerNode = null; - minSplitSizePerRack = null; - } - - public String getCommand() { - return command; - } + private boolean finalMapRed; - public void setCommand(final String command) { - this.command = command; - } - - @Explain(displayName = "Path -> Alias", normalExplain = false) - public LinkedHashMap> getPathToAliases() { - return pathToAliases; - } - - public void setPathToAliases( - final LinkedHashMap> pathToAliases) { - this.pathToAliases = pathToAliases; - } - - @Explain(displayName = "Truncated Path -> Alias", normalExplain = false) - /** - * This is used to display and verify output of "Path -> Alias" in test framework. - * - * {@link QTestUtil} masks "Path -> Alias" and makes verification impossible. - * By keeping "Path -> Alias" intact and adding a new display name which is not - * masked by {@link QTestUtil} by removing prefix. - * - * Notes: we would still be masking for intermediate directories. - * - * @return - */ - public Map> getTruncatedPathToAliases() { - Map> trunPathToAliases = new LinkedHashMap>(); - Iterator>> itr = this.pathToAliases.entrySet().iterator(); - while (itr.hasNext()) { - final Entry> entry = itr.next(); - String origiKey = entry.getKey(); - String newKey = PlanUtils.removePrefixFromWarehouseConfig(origiKey); - ArrayList value = entry.getValue(); - trunPathToAliases.put(newKey, value); - } - return trunPathToAliases; - } - - - - @Explain(displayName = "Path -> Partition", normalExplain = false) - public LinkedHashMap getPathToPartitionInfo() { - return pathToPartitionInfo; - } - - public void setPathToPartitionInfo( - final LinkedHashMap pathToPartitionInfo) { - this.pathToPartitionInfo = pathToPartitionInfo; - } - - /** - * @return the aliasToPartnInfo - */ - public LinkedHashMap getAliasToPartnInfo() { - return aliasToPartnInfo; - } - - /** - * @param aliasToPartnInfo - * the aliasToPartnInfo to set - */ - public void setAliasToPartnInfo( - LinkedHashMap aliasToPartnInfo) { - this.aliasToPartnInfo = aliasToPartnInfo; - } - - @Explain(displayName = "Alias -> Map Operator Tree") - public LinkedHashMap> getAliasToWork() { - return aliasToWork; - } - - public void setAliasToWork( - final LinkedHashMap> aliasToWork) { - this.aliasToWork = aliasToWork; - } - - public void mergeAliasedInput(String alias, String pathDir, PartitionDesc partitionInfo) { - ArrayList aliases = pathToAliases.get(pathDir); - if (aliases == null) { - aliases = new ArrayList(Arrays.asList(alias)); - pathToAliases.put(pathDir, aliases); - pathToPartitionInfo.put(pathDir, partitionInfo); - } else { - aliases.add(alias); - } + @Explain(skipHeader = true, displayName = "Map") + public MapWork getMapWork() { + return mapWork; } - public ArrayList getAliases() { - return new ArrayList(aliasToWork.keySet()); + public void setMapWork(MapWork mapWork) { + this.mapWork = mapWork; } - public ArrayList> getWorks() { - return new ArrayList>(aliasToWork.values()); + @Explain(skipHeader = true, displayName = "Reduce") + public ReduceWork getReduceWork() { + return reduceWork; } - public ArrayList getPaths() { - return new ArrayList(pathToAliases.keySet()); + public void setReduceWork(ReduceWork reduceWork) { + this.reduceWork = reduceWork; } - public ArrayList getPartitionDescs() { - return new ArrayList(aliasToPartnInfo.values()); - } - - /** - * @return the mapredLocalWork - */ - @Explain(displayName = "Local Work") - public MapredLocalWork getMapLocalWork() { - return mapLocalWork; - } - - /** - * @param mapLocalWork - * the mapredLocalWork to set - */ - public void setMapLocalWork(final MapredLocalWork mapLocalWork) { - this.mapLocalWork = mapLocalWork; - } - - public TableDesc getKeyDesc() { - return keyDesc; - } - - /** - * If the plan has a reducer and correspondingly a reduce-sink, then store the TableDesc pointing - * to keySerializeInfo of the ReduceSink - * - * @param keyDesc - */ - public void setKeyDesc(final TableDesc keyDesc) { - this.keyDesc = keyDesc; - } - - public List getTagToValueDesc() { - return tagToValueDesc; - } - - public void setTagToValueDesc(final List tagToValueDesc) { - this.tagToValueDesc = tagToValueDesc; - } - - @Explain(displayName = "Reduce Operator Tree") - public Operator getReducer() { - return reducer; - } - - @Explain(displayName = "Split Sample") - public HashMap getNameToSplitSample() { - return nameToSplitSample; - } - - public void setNameToSplitSample(HashMap nameToSplitSample) { - this.nameToSplitSample = nameToSplitSample; - } - - public void setReducer(final Operator reducer) { - this.reducer = reducer; - } - - public Integer getNumMapTasks() { - return numMapTasks; - } - - public void setNumMapTasks(Integer numMapTasks) { - this.numMapTasks = numMapTasks; - } - - /** - * If the number of reducers is -1, the runtime will automatically figure it - * out by input data size. - * - * The number of reducers will be a positive number only in case the target - * table is bucketed into N buckets (through CREATE TABLE). This feature is - * not supported yet, so the number of reducers will always be -1 for now. - */ - public Integer getNumReduceTasks() { - return numReduceTasks; - } - - public void setNumReduceTasks(final Integer numReduceTasks) { - this.numReduceTasks = numReduceTasks; - } - - @Explain(displayName = "Path -> Bucketed Columns", normalExplain = false) - public Map> getBucketedColsByDirectory() { - return bucketedColsByDirectory; + public boolean isFinalMapRed() { + return finalMapRed; } - @Explain(displayName = "Path -> Sorted Columns", normalExplain = false) - public Map> getSortedColsByDirectory() { - return sortedColsByDirectory; + public void setFinalMapRed(boolean finalMapRed) { + this.finalMapRed = finalMapRed; } - @SuppressWarnings("nls") - public void addMapWork(String path, String alias, Operator work, - PartitionDesc pd) { - ArrayList curAliases = pathToAliases.get(path); - if (curAliases == null) { - assert (pathToPartitionInfo.get(path) == null); - curAliases = new ArrayList(); - pathToAliases.put(path, curAliases); - pathToPartitionInfo.put(path, pd); - } else { - assert (pathToPartitionInfo.get(path) != null); - } - - for (String oneAlias : curAliases) { - if (oneAlias.equals(alias)) { - throw new RuntimeException("Multiple aliases named: " + alias - + " for path: " + path); - } + public void configureJobConf(JobConf job) { + mapWork.configureJobConf(job); + if (reduceWork != null) { + reduceWork.configureJobConf(job); } - curAliases.add(alias); - - if (aliasToWork.get(alias) != null) { - throw new RuntimeException("Existing work for alias: " + alias); - } - aliasToWork.put(alias, work); } - @SuppressWarnings("nls") - public String isInvalid() { - if ((getNumReduceTasks() >= 1) && (getReducer() == null)) { - return "Reducers > 0 but no reduce operator"; - } - - if ((getNumReduceTasks() == 0) && (getReducer() != null)) { - return "Reducers == 0 but reduce operator specified"; + public List> getAllOperators() { + List> ops = new ArrayList>(); + ops.addAll(mapWork.getAllOperators()); + if (reduceWork != null) { + ops.addAll(reduceWork.getAllOperators()); } - return null; + return ops; } public String toXML() { @@ -393,241 +89,4 @@ public String toXML() { return (baos.toString()); } - // non bean - - /** - * For each map side operator - stores the alias the operator is working on - * behalf of in the operator runtime state. This is used by reducesink - * operator - but could be useful for debugging as well. - */ - private void setAliases() { - if(aliasToWork == null) { - return; - } - for (String oneAlias : aliasToWork.keySet()) { - aliasToWork.get(oneAlias).setAlias(oneAlias); - } - } - - /** - * Derive additional attributes to be rendered by EXPLAIN. - */ - public void deriveExplainAttributes() { - if (pathToPartitionInfo != null) { - for (Map.Entry entry : pathToPartitionInfo - .entrySet()) { - entry.getValue().deriveBaseFileName(entry.getKey()); - } - } - if (mapLocalWork != null) { - mapLocalWork.deriveExplainAttributes(); - } - } - - public void initialize() { - setAliases(); - } - - @Explain(displayName = "Needs Tagging", normalExplain = false) - public boolean getNeedsTagging() { - return needsTagging; - } - - public void setNeedsTagging(boolean needsTagging) { - this.needsTagging = needsTagging; - } - - public boolean getHadoopSupportsSplittable() { - return hadoopSupportsSplittable; - } - - public void setHadoopSupportsSplittable(boolean hadoopSupportsSplittable) { - this.hadoopSupportsSplittable = hadoopSupportsSplittable; - } - - public Long getMaxSplitSize() { - return maxSplitSize; - } - - public void setMaxSplitSize(Long maxSplitSize) { - this.maxSplitSize = maxSplitSize; - } - - public Long getMinSplitSize() { - return minSplitSize; - } - - public void setMinSplitSize(Long minSplitSize) { - this.minSplitSize = minSplitSize; - } - - public Long getMinSplitSizePerNode() { - return minSplitSizePerNode; - } - - public void setMinSplitSizePerNode(Long minSplitSizePerNode) { - this.minSplitSizePerNode = minSplitSizePerNode; - } - - public Long getMinSplitSizePerRack() { - return minSplitSizePerRack; - } - - public void setMinSplitSizePerRack(Long minSplitSizePerRack) { - this.minSplitSizePerRack = minSplitSizePerRack; - } - - public String getInputformat() { - return inputformat; - } - - public void setInputformat(String inputformat) { - this.inputformat = inputformat; - } - - public String getIndexIntermediateFile() { - return indexIntermediateFile; - } - - public void addIndexIntermediateFile(String fileName) { - if (this.indexIntermediateFile == null) { - this.indexIntermediateFile = fileName; - } else { - this.indexIntermediateFile += "," + fileName; - } - } - - public void setGatheringStats(boolean gatherStats) { - this.gatheringStats = gatherStats; - } - - public boolean isGatheringStats() { - return this.gatheringStats; - } - - public void setMapperCannotSpanPartns(boolean mapperCannotSpanPartns) { - this.mapperCannotSpanPartns = mapperCannotSpanPartns; - } - - public boolean isMapperCannotSpanPartns() { - return this.mapperCannotSpanPartns; - } - - public String getTmpHDFSFileURI() { - return tmpHDFSFileURI; - } - - public void setTmpHDFSFileURI(String tmpHDFSFileURI) { - this.tmpHDFSFileURI = tmpHDFSFileURI; - } - - - public QBJoinTree getJoinTree() { - return joinTree; - } - - public void setJoinTree(QBJoinTree joinTree) { - this.joinTree = joinTree; - } - - public - LinkedHashMap, OpParseContext> getOpParseCtxMap() { - return opParseCtxMap; - } - - public void setOpParseCtxMap( - LinkedHashMap, OpParseContext> opParseCtxMap) { - this.opParseCtxMap = opParseCtxMap; - } - - public boolean isInputFormatSorted() { - return inputFormatSorted; - } - - public void setInputFormatSorted(boolean inputFormatSorted) { - this.inputFormatSorted = inputFormatSorted; - } - - public void resolveDynamicPartitionStoredAsSubDirsMerge(HiveConf conf, Path path, - TableDesc tblDesc, ArrayList aliases, PartitionDesc partDesc) { - pathToAliases.put(path.toString(), aliases); - pathToPartitionInfo.put(path.toString(), partDesc); - } - - public List> getAllOperators() { - ArrayList> opList = new ArrayList>(); - ArrayList> returnList = new ArrayList>(); - - if (getReducer() != null) { - opList.add(getReducer()); - } - - Map> pa = getPathToAliases(); - if (pa != null) { - for (List ls : pa.values()) { - for (String a : ls) { - Operator op = getAliasToWork().get(a); - if (op != null ) { - opList.add(op); - } - } - } - } - - //recursively add all children - while (!opList.isEmpty()) { - Operator op = opList.remove(0); - if (op.getChildOperators() != null) { - opList.addAll(op.getChildOperators()); - } - returnList.add(op); - } - - return returnList; - } - - public boolean isUseBucketizedHiveInputFormat() { - return useBucketizedHiveInputFormat; - } - - public void setUseBucketizedHiveInputFormat(boolean useBucketizedHiveInputFormat) { - this.useBucketizedHiveInputFormat = useBucketizedHiveInputFormat; - } - - public boolean isFinalMapRed() { - return finalMapRed; - } - - public void setFinalMapRed(boolean finalMapRed) { - this.finalMapRed = finalMapRed; - } - - public void configureJobConf(JobConf jobConf) { - for (PartitionDesc partition : aliasToPartnInfo.values()) { - PlanUtils.configureJobConf(partition.getTableDesc(), jobConf); - } - Collection> mappers = aliasToWork.values(); - for (FileSinkOperator fs : OperatorUtils.findOperators(mappers, FileSinkOperator.class)) { - PlanUtils.configureJobConf(fs.getConf().getTableInfo(), jobConf); - } - if (reducer != null) { - for (FileSinkOperator fs : OperatorUtils.findOperators(reducer, FileSinkOperator.class)) { - PlanUtils.configureJobConf(fs.getConf().getTableInfo(), jobConf); - } - } - } - - public int getSamplingType() { - return samplingType; - } - - public void setSamplingType(int samplingType) { - this.samplingType = samplingType; - } - - @Explain(displayName = "Sampling") - public String getSamplingTypeString() { - return samplingType == 1 ? "SAMPLING_ON_PREV_MR" : - samplingType == 2 ? "SAMPLING_ON_START" : null; - } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java index edd62c8..5f983db 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java @@ -279,7 +279,7 @@ public PartitionDesc clone() { * @param path * URI to the partition file */ - void deriveBaseFileName(String path) { + public void deriveBaseFileName(String path) { PlanUtils.configureInputJobPropertiesForStorageHandler(tableDesc); if (path == null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java index 89964f0..5fd8d828 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.exec.ColumnInfo; -import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.RowSchema; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.hooks.ReadEntity; @@ -89,12 +88,10 @@ public static long getCountForMapJoinDumpFilePrefix() { @SuppressWarnings("nls") public static MapredWork getMapRedWork() { try { - return new MapredWork("", new LinkedHashMap>(), - new LinkedHashMap(), - new LinkedHashMap>(), - new TableDesc(), new ArrayList(), null, Integer.valueOf(1), - null, Hive.get().getConf().getBoolVar( + MapredWork work = new MapredWork(); + work.getMapWork().setHadoopSupportsSplittable(Hive.get().getConf().getBoolVar( HiveConf.ConfVars.HIVE_COMBINE_INPUT_FORMAT_SUPPORTS_SPLITTABLE)); + return work; } catch (HiveException ex) { throw new RuntimeException(ex); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java new file mode 100644 index 0000000..5da9416 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorUtils; +import org.apache.hadoop.mapred.JobConf; +import org.apache.log4j.Logger; + +@SuppressWarnings({"serial", "deprecation"}) +public class ReduceWork extends BaseWork { + Logger LOG = Logger.getLogger(ReduceWork.class); + + // schema of the map-reduce 'key' object - this is homogeneous + private TableDesc keyDesc; + + // schema of the map-reduce 'value' object - this is heterogeneous + private List tagToValueDesc = new ArrayList(); + private Operator reducer; + private Integer numReduceTasks; + + private boolean needsTagging; + + /** + * If the plan has a reducer and correspondingly a reduce-sink, then store the TableDesc pointing + * to keySerializeInfo of the ReduceSink + * + * @param keyDesc + */ + public void setKeyDesc(final TableDesc keyDesc) { + this.keyDesc = keyDesc; + } + + public TableDesc getKeyDesc() { + return keyDesc; + } + + public List getTagToValueDesc() { + return tagToValueDesc; + } + + public void setTagToValueDesc(final List tagToValueDesc) { + this.tagToValueDesc = tagToValueDesc; + } + + @Explain(displayName = "Reduce Operator Tree") + public Operator getReducer() { + return reducer; + } + + public void setReducer(final Operator reducer) { + this.reducer = reducer; + } + + @Explain(displayName = "Needs Tagging", normalExplain = false) + public boolean getNeedsTagging() { + return needsTagging; + } + + public void setNeedsTagging(boolean needsTagging) { + this.needsTagging = needsTagging; + } + + @Override + protected List> getAllRootOperators() { + ArrayList> opList = new ArrayList>(); + opList.add(getReducer()); + return opList; + } + + /** + * If the number of reducers is -1, the runtime will automatically figure it + * out by input data size. + * + * The number of reducers will be a positive number only in case the target + * table is bucketed into N buckets (through CREATE TABLE). This feature is + * not supported yet, so the number of reducers will always be -1 for now. + */ + public Integer getNumReduceTasks() { + return numReduceTasks; + } + + public void setNumReduceTasks(final Integer numReduceTasks) { + this.numReduceTasks = numReduceTasks; + } + + public void configureJobConf(JobConf job) { + if (reducer != null) { + for (FileSinkOperator fs : OperatorUtils.findOperators(reducer, FileSinkOperator.class)) { + PlanUtils.configureJobConf(fs.getConf().getTableInfo(), job); + } + } + } +} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java index 6a74ae4..4371589 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java @@ -49,6 +49,7 @@ 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.ReduceWork; import org.apache.hadoop.hive.ql.plan.ScriptDesc; import org.apache.hadoop.hive.ql.plan.SelectDesc; import org.apache.hadoop.hive.serde.serdeConstants; @@ -136,7 +137,7 @@ protected void setUp() { } public static void addMapWork(MapredWork mr, Table tbl, String alias, Operator work) { - mr.addMapWork(tbl.getDataLocation().toString(), alias, work, new PartitionDesc( + mr.getMapWork().addMapWork(tbl.getDataLocation().toString(), alias, work, new PartitionDesc( Utilities.getTableDesc(tbl), null)); } @@ -191,7 +192,6 @@ private FilterDesc getTestFilterDesc(String column) { @SuppressWarnings("unchecked") private void populateMapPlan1(Table src) { - mr.setNumReduceTasks(Integer.valueOf(0)); Operator op2 = OperatorFactory.get(new FileSinkDesc(tmpdir + "mapplan1.out", Utilities.defaultTd, true)); @@ -203,7 +203,6 @@ private void populateMapPlan1(Table src) { @SuppressWarnings("unchecked") private void populateMapPlan2(Table src) { - mr.setNumReduceTasks(Integer.valueOf(0)); Operator op3 = OperatorFactory.get(new FileSinkDesc(tmpdir + "mapplan2.out", Utilities.defaultTd, false)); @@ -222,7 +221,6 @@ private void populateMapPlan2(Table src) { @SuppressWarnings("unchecked") private void populateMapRedPlan1(Table src) throws SemanticException { - mr.setNumReduceTasks(Integer.valueOf(1)); ArrayList outputColumns = new ArrayList(); for (int i = 0; i < 2; i++) { @@ -235,8 +233,11 @@ private void populateMapRedPlan1(Table src) throws SemanticException { -1, 1, -1)); addMapWork(mr, src, "a", op1); - mr.setKeyDesc(op1.getConf().getKeySerializeInfo()); - mr.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + ReduceWork rWork = new ReduceWork(); + rWork.setNumReduceTasks(Integer.valueOf(1)); + rWork.setKeyDesc(op1.getConf().getKeySerializeInfo()); + rWork.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + mr.setReduceWork(rWork); // reduce side work Operator op3 = OperatorFactory.get(new FileSinkDesc(tmpdir @@ -245,12 +246,11 @@ private void populateMapRedPlan1(Table src) throws SemanticException { Operator op2 = OperatorFactory.get(new ExtractDesc( getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); - mr.setReducer(op2); + rWork.setReducer(op2); } @SuppressWarnings("unchecked") private void populateMapRedPlan2(Table src) throws SemanticException { - mr.setNumReduceTasks(Integer.valueOf(1)); ArrayList outputColumns = new ArrayList(); for (int i = 0; i < 2; i++) { outputColumns.add("_col" + i); @@ -263,8 +263,11 @@ private void populateMapRedPlan2(Table src) throws SemanticException { outputColumns, false, -1, 1, -1)); addMapWork(mr, src, "a", op1); - mr.setKeyDesc(op1.getConf().getKeySerializeInfo()); - mr.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + ReduceWork rWork = new ReduceWork(); + rWork.setNumReduceTasks(Integer.valueOf(1)); + rWork.setKeyDesc(op1.getConf().getKeySerializeInfo()); + rWork.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + mr.setReduceWork(rWork); // reduce side work Operator op4 = OperatorFactory.get(new FileSinkDesc(tmpdir @@ -275,7 +278,7 @@ private void populateMapRedPlan2(Table src) throws SemanticException { Operator op2 = OperatorFactory.get(new ExtractDesc( getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); - mr.setReducer(op2); + rWork.setReducer(op2); } /** @@ -283,8 +286,6 @@ private void populateMapRedPlan2(Table src) throws SemanticException { */ @SuppressWarnings("unchecked") private void populateMapRedPlan3(Table src, Table src2) throws SemanticException { - mr.setNumReduceTasks(Integer.valueOf(5)); - mr.setNeedsTagging(true); ArrayList outputColumns = new ArrayList(); for (int i = 0; i < 2; i++) { outputColumns.add("_col" + i); @@ -296,8 +297,6 @@ private void populateMapRedPlan3(Table src, Table src2) throws SemanticException Byte.valueOf((byte) 0), 1, -1)); addMapWork(mr, src, "a", op1); - mr.setKeyDesc(op1.getConf().getKeySerializeInfo()); - mr.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); Operator op2 = OperatorFactory.get(PlanUtils .getReduceSinkDesc(Utilities.makeList(getStringColumn("key")), @@ -305,7 +304,14 @@ private void populateMapRedPlan3(Table src, Table src2) throws SemanticException Byte.valueOf((byte) 1), Integer.MAX_VALUE, -1)); addMapWork(mr, src2, "b", op2); - mr.getTagToValueDesc().add(op2.getConf().getValueSerializeInfo()); + ReduceWork rWork = new ReduceWork(); + rWork.setNumReduceTasks(Integer.valueOf(5)); + rWork.setNeedsTagging(true); + rWork.setKeyDesc(op1.getConf().getKeySerializeInfo()); + rWork.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + + mr.setReduceWork(rWork); + rWork.getTagToValueDesc().add(op2.getConf().getValueSerializeInfo()); // reduce side work Operator op4 = OperatorFactory.get(new FileSinkDesc(tmpdir @@ -319,12 +325,11 @@ private void populateMapRedPlan3(Table src, Table src2) throws SemanticException Utilities.ReduceField.VALUE.toString(), "", false), "0", false)), outputColumns), op4); - mr.setReducer(op5); + rWork.setReducer(op5); } @SuppressWarnings("unchecked") private void populateMapRedPlan4(Table src) throws SemanticException { - mr.setNumReduceTasks(Integer.valueOf(1)); // map-side work ArrayList outputColumns = new ArrayList(); @@ -347,8 +352,11 @@ private void populateMapRedPlan4(Table src) throws SemanticException { outputColumns), op0); addMapWork(mr, src, "a", op4); - mr.setKeyDesc(op1.getConf().getKeySerializeInfo()); - mr.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + ReduceWork rWork = new ReduceWork(); + rWork.setKeyDesc(op1.getConf().getKeySerializeInfo()); + rWork.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + rWork.setNumReduceTasks(Integer.valueOf(1)); + mr.setReduceWork(rWork); // reduce side work Operator op3 = OperatorFactory.get(new FileSinkDesc(tmpdir @@ -357,7 +365,7 @@ private void populateMapRedPlan4(Table src) throws SemanticException { Operator op2 = OperatorFactory.get(new ExtractDesc( getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); - mr.setReducer(op2); + rWork.setReducer(op2); } public static ExprNodeColumnDesc getStringColumn(String columnName) { @@ -367,7 +375,6 @@ public static ExprNodeColumnDesc getStringColumn(String columnName) { @SuppressWarnings("unchecked") private void populateMapRedPlan5(Table src) throws SemanticException { - mr.setNumReduceTasks(Integer.valueOf(1)); // map-side work ArrayList outputColumns = new ArrayList(); @@ -384,8 +391,11 @@ private void populateMapRedPlan5(Table src) throws SemanticException { outputColumns), op0); addMapWork(mr, src, "a", op4); - mr.setKeyDesc(op0.getConf().getKeySerializeInfo()); - mr.getTagToValueDesc().add(op0.getConf().getValueSerializeInfo()); + ReduceWork rWork = new ReduceWork(); + mr.setReduceWork(rWork); + rWork.setNumReduceTasks(Integer.valueOf(1)); + rWork.setKeyDesc(op0.getConf().getKeySerializeInfo()); + rWork.getTagToValueDesc().add(op0.getConf().getValueSerializeInfo()); // reduce side work Operator op3 = OperatorFactory.get(new FileSinkDesc(tmpdir @@ -394,12 +404,11 @@ private void populateMapRedPlan5(Table src) throws SemanticException { Operator op2 = OperatorFactory.get(new ExtractDesc( getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); - mr.setReducer(op2); + rWork.setReducer(op2); } @SuppressWarnings("unchecked") private void populateMapRedPlan6(Table src) throws SemanticException { - mr.setNumReduceTasks(Integer.valueOf(1)); // map-side work ArrayList outputColumns = new ArrayList(); @@ -423,8 +432,11 @@ private void populateMapRedPlan6(Table src) throws SemanticException { outputColumns), op0); addMapWork(mr, src, "a", op4); - mr.setKeyDesc(op1.getConf().getKeySerializeInfo()); - mr.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); + ReduceWork rWork = new ReduceWork(); + mr.setReduceWork(rWork); + rWork.setNumReduceTasks(Integer.valueOf(1)); + rWork.setKeyDesc(op1.getConf().getKeySerializeInfo()); + rWork.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo()); // reduce side work Operator op3 = OperatorFactory.get(new FileSinkDesc(tmpdir @@ -435,7 +447,7 @@ private void populateMapRedPlan6(Table src) throws SemanticException { Operator op5 = OperatorFactory.get(new ExtractDesc( getStringColumn(Utilities.ReduceField.VALUE.toString())), op2); - mr.setReducer(op5); + rWork.setReducer(op5); } private void executePlan() throws Exception { diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java index 79bed09..fac30bf 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java @@ -345,13 +345,13 @@ public void testMapOperator() throws Throwable { // initialize mapredWork MapredWork mrwork = new MapredWork(); - mrwork.setPathToAliases(pathToAliases); - mrwork.setPathToPartitionInfo(pathToPartitionInfo); - mrwork.setAliasToWork(aliasToWork); + mrwork.getMapWork().setPathToAliases(pathToAliases); + mrwork.getMapWork().setPathToPartitionInfo(pathToPartitionInfo); + mrwork.getMapWork().setAliasToWork(aliasToWork); // get map operator and initialize it MapOperator mo = new MapOperator(); - mo.initializeAsRoot(hconf, mrwork); + mo.initializeAsRoot(hconf, mrwork.getMapWork()); Text tw = new Text(); InspectableObject io1 = new InspectableObject(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java index 8ec50d7..2c3e97b 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java @@ -75,9 +75,9 @@ public void testPlan() throws Exception { ao.put("a", op); MapredWork mrwork = new MapredWork(); - mrwork.setPathToAliases(pa); - mrwork.setPathToPartitionInfo(pt); - mrwork.setAliasToWork(ao); + mrwork.getMapWork().setPathToAliases(pa); + mrwork.getMapWork().setPathToPartitionInfo(pt); + mrwork.getMapWork().setAliasToWork(ao); // serialize the configuration once .. ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHiveSortedInputFormatUsedHook.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHiveSortedInputFormatUsedHook.java index 4bb7801..a64086b 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHiveSortedInputFormatUsedHook.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHiveSortedInputFormatUsedHook.java @@ -38,7 +38,7 @@ public void run(HookContext hookContext) { for (Task rootTask : rootTasks) { if (rootTask.getWork() instanceof MapredWork) { Assert.assertTrue("The root map reduce task's input was not marked as sorted.", - ((MapredWork)rootTask.getWork()).isInputFormatSorted()); + ((MapredWork)rootTask.getWork()).getMapWork().isInputFormatSorted()); } } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java index ad231c5..edd739a 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java @@ -46,7 +46,6 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.util.ReflectionUtils; /** @@ -97,31 +96,31 @@ public void testCombine() throws Exception { JobConf newJob = new JobConf(job); FileSystem fs = dataDir1.getFileSystem(newJob); int symbolLinkedFileSize = 0; - + Path dir1_file1 = new Path(dataDir1, "combinefile1_1"); writeTextFile(dir1_file1, "dir1_file1_line1\n" + "dir1_file1_line2\n"); - + symbolLinkedFileSize += fs.getFileStatus(dir1_file1).getLen(); - + Path dir2_file1 = new Path(dataDir2, "combinefile2_1"); writeTextFile(dir2_file1, "dir2_file1_line1\n" + "dir2_file1_line2\n"); - + symbolLinkedFileSize += fs.getFileStatus(dir2_file1).getLen(); - + // A symlink file, contains first file from first dir and second file from // second dir. writeSymlinkFile( new Path(symlinkDir, "symlink_file"), new Path(dataDir1, "combinefile1_1"), new Path(dataDir2, "combinefile2_1")); - - + + HiveConf hiveConf = new HiveConf(TestSymlinkTextInputFormat.class); - + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_REWORK_MAPREDWORK, true); HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); Driver drv = new Driver(hiveConf); @@ -131,9 +130,9 @@ public void testCombine() throws Exception { String createSymlinkTableCmd = "create table " + tblName + " (key int) stored as " + " inputformat 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' " + " outputformat 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'"; - + SessionState.start(hiveConf); - + boolean tblCreated = false; try { int ecode = 0; @@ -144,15 +143,15 @@ public void testCombine() throws Exception { } tblCreated = true; - String loadFileCommand = "LOAD DATA LOCAL INPATH '" + + String loadFileCommand = "LOAD DATA LOCAL INPATH '" + new Path(symlinkDir, "symlink_file").toString() + "' INTO TABLE " + tblName; - + ecode = drv.run(loadFileCommand).getResponseCode(); if (ecode != 0) { throw new Exception("Load data command: " + loadFileCommand + " failed with exit code= " + ecode); } - + String cmd = "select key from " + tblName; drv.compile(cmd); @@ -164,23 +163,23 @@ public void testCombine() throws Exception { emptyScratchDir = new Path(emptyScratchDirStr); FileSystem fileSys = emptyScratchDir.getFileSystem(newJob); fileSys.mkdirs(emptyScratchDir); - + QueryPlan plan = drv.getPlan(); MapRedTask selectTask = (MapRedTask)plan.getRootTasks().get(0); - - ExecDriver.addInputPaths(newJob, selectTask.getWork(), emptyScratchDir.toString(), ctx); + + ExecDriver.addInputPaths(newJob, selectTask.getWork().getMapWork(), emptyScratchDir.toString(), ctx); Utilities.setMapRedWork(newJob, selectTask.getWork(), ctx.getMRTmpFileURI()); - + CombineHiveInputFormat combineInputFormat = ReflectionUtils.newInstance( CombineHiveInputFormat.class, newJob); - + combineInputFormat.validateInput(newJob); - + InputSplit[] retSplits = combineInputFormat.getSplits(newJob, 1); assertEquals(1, retSplits.length); } catch (Exception e) { e.printStackTrace(); - fail("Caught exception " + e); + fail("Caught exception " + e); } finally { if (tblCreated) { drv.run("drop table text_symlink_text").getResponseCode(); @@ -194,48 +193,48 @@ public void testCombine() throws Exception { */ public void testAccuracy1() throws IOException { // First data dir, contains 2 files. - + FileSystem fs = dataDir1.getFileSystem(job); int symbolLinkedFileSize = 0; - + Path dir1_file1 = new Path(dataDir1, "file1"); writeTextFile(dir1_file1, "dir1_file1_line1\n" + "dir1_file1_line2\n"); - + symbolLinkedFileSize += fs.getFileStatus(dir1_file1).getLen(); - + Path dir1_file2 = new Path(dataDir1, "file2"); writeTextFile(dir1_file2, "dir1_file2_line1\n" + "dir1_file2_line2\n"); - + // Second data dir, contains 2 files. - + Path dir2_file1 = new Path(dataDir2, "file1"); writeTextFile(dir2_file1, "dir2_file1_line1\n" + "dir2_file1_line2\n"); - + Path dir2_file2 = new Path(dataDir2, "file2"); writeTextFile(dir2_file2, "dir2_file2_line1\n" + "dir2_file2_line2\n"); symbolLinkedFileSize += fs.getFileStatus(dir2_file2).getLen(); - + // A symlink file, contains first file from first dir and second file from // second dir. writeSymlinkFile( new Path(symlinkDir, "symlink_file"), new Path(dataDir1, "file1"), new Path(dataDir2, "file2")); - + SymlinkTextInputFormat inputFormat = new SymlinkTextInputFormat(); - + //test content summary ContentSummary cs = inputFormat.getContentSummary(symlinkDir, job); - + assertEquals(symbolLinkedFileSize, cs.getLength()); assertEquals(2, cs.getFileCount()); assertEquals(0, cs.getDirectoryCount()); @@ -279,13 +278,13 @@ public void testAccuracy2() throws IOException { FileInputFormat.setInputPaths(job, symlinkDir); SymlinkTextInputFormat inputFormat = new SymlinkTextInputFormat(); - + ContentSummary cs = inputFormat.getContentSummary(symlinkDir, job); - + assertEquals(0, cs.getLength()); assertEquals(0, cs.getFileCount()); assertEquals(0, cs.getDirectoryCount()); - + InputSplit[] splits = inputFormat.getSplits(job, 2); log.info("Number of splits: " + splits.length); diff --git a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out index 2720736..f71d482 100644 --- a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out +++ b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out @@ -126,7 +126,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -176,6 +175,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.alter_coltype name: default.alter_coltype + Truncated Path -> Alias: + /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -206,8 +208,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] Stage: Stage-0 Fetch Operator @@ -315,7 +315,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -365,6 +364,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.alter_coltype name: default.alter_coltype + Truncated Path -> Alias: + /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -395,8 +397,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] Stage: Stage-0 Fetch Operator @@ -471,7 +471,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -613,6 +612,11 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.alter_coltype name: default.alter_coltype + Truncated Path -> Alias: + /alter_coltype/dt=10/ts=3.0 [alter_coltype] + /alter_coltype/dt=100x/ts=3.0 [alter_coltype] + /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -643,10 +647,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /alter_coltype/dt=10/ts=3.0 [alter_coltype] - /alter_coltype/dt=100x/ts=3.0 [alter_coltype] - /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] Stage: Stage-0 Fetch Operator @@ -804,7 +804,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1038,7 +1037,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1180,6 +1178,11 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.alter_coltype name: default.alter_coltype + Truncated Path -> Alias: + /alter_coltype/dt=10/ts=3.0 [alter_coltype] + /alter_coltype/dt=100x/ts=3.0 [alter_coltype] + /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -1210,10 +1213,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /alter_coltype/dt=10/ts=3.0 [alter_coltype] - /alter_coltype/dt=100x/ts=3.0 [alter_coltype] - /alter_coltype/dt=100x/ts=6%3A30pm [alter_coltype] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out index 0e8f96b..14006c6 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out @@ -122,7 +122,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -222,6 +221,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -252,9 +255,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -321,7 +321,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -421,6 +420,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -451,9 +454,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator @@ -551,7 +551,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -698,6 +697,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -728,9 +731,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-7 Map Reduce Local Work @@ -786,7 +786,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -933,6 +932,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -963,8 +965,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] Stage: Stage-1 Map Reduce @@ -997,7 +997,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1097,6 +1096,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -1127,9 +1130,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out index 42e25fa..9769bd8 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out @@ -152,7 +152,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -311,7 +310,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -331,6 +329,9 @@ STAGE PLANS: columns _col0 columns.types bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -361,8 +362,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Fetch Operator @@ -471,7 +470,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -630,7 +628,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -650,6 +647,9 @@ STAGE PLANS: columns _col0 columns.types bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -680,8 +680,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Fetch Operator @@ -782,7 +780,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -880,6 +877,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -910,9 +911,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out index 47a8635..b93b6b4 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out @@ -236,7 +236,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -446,7 +445,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -466,6 +464,9 @@ STAGE PLANS: columns _col0 columns.types bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -496,8 +497,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out index c39d057..8de34df 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out @@ -106,7 +106,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -206,6 +205,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -236,9 +239,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator @@ -338,7 +338,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -485,6 +484,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -515,9 +518,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-7 Map Reduce Local Work @@ -573,7 +573,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -720,6 +719,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -750,8 +752,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] Stage: Stage-1 Map Reduce @@ -784,7 +784,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -884,6 +883,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -914,9 +917,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out index bd7381f..d9840d6 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out @@ -106,7 +106,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -158,6 +157,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -188,8 +190,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] Stage: Stage-0 Fetch Operator @@ -256,7 +256,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -308,6 +307,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -338,8 +340,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] Stage: Stage-0 Fetch Operator @@ -437,7 +437,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -583,6 +582,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -613,8 +615,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] Stage: Stage-7 Map Reduce Local Work @@ -670,7 +670,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -816,6 +815,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [b] + /bucket_small/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -846,9 +849,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] - /bucket_small/ds=2008-04-09 [b] Stage: Stage-1 Map Reduce @@ -881,7 +881,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -933,6 +932,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -963,8 +965,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out index 6121722..e3c15c9 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out @@ -118,7 +118,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -170,6 +169,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -200,8 +202,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] Stage: Stage-0 Fetch Operator @@ -268,7 +268,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -320,6 +319,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -350,8 +352,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] Stage: Stage-0 Fetch Operator @@ -449,7 +449,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -595,6 +594,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -625,8 +627,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] Stage: Stage-7 Map Reduce Local Work @@ -682,7 +682,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -828,6 +827,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [b] + /bucket_small/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -858,9 +861,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] - /bucket_small/ds=2008-04-09 [b] Stage: Stage-1 Map Reduce @@ -893,7 +893,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -945,6 +944,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -975,8 +977,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out index c3ea09d..00a985c 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out @@ -91,7 +91,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -141,6 +140,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -171,8 +173,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big [b] Stage: Stage-0 Fetch Operator @@ -233,7 +233,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -283,6 +282,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -313,8 +315,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big [a] Stage: Stage-0 Fetch Operator @@ -406,7 +406,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -494,6 +493,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_big [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -524,8 +526,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big [a] Stage: Stage-7 Map Reduce Local Work @@ -581,7 +581,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -669,6 +668,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_small [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -699,8 +701,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_small [b] Stage: Stage-1 Map Reduce @@ -733,7 +733,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -783,6 +782,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -813,8 +815,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out index e0cd848..5d3119b 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out @@ -131,7 +131,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -231,6 +230,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -261,9 +264,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -332,7 +332,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -432,6 +431,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -462,9 +465,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator @@ -564,7 +564,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -758,6 +757,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -788,9 +791,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-7 Map Reduce Local Work @@ -846,7 +846,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1040,6 +1039,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [b] + /bucket_small/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -1070,9 +1073,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] - /bucket_small/ds=2008-04-09 [b] Stage: Stage-1 Map Reduce @@ -1105,7 +1105,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1205,6 +1204,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -1235,9 +1238,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out index 924fbad..e363379 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out @@ -131,7 +131,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -231,6 +230,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -261,9 +264,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -332,7 +332,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -432,6 +431,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -462,9 +465,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator @@ -566,7 +566,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -760,6 +759,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -790,9 +793,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-7 Map Reduce Local Work @@ -848,7 +848,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1042,6 +1041,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_small name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [b] + /bucket_small/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -1072,9 +1075,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] - /bucket_small/ds=2008-04-09 [b] Stage: Stage-1 Map Reduce @@ -1107,7 +1107,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1207,6 +1206,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -1237,9 +1240,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/binary_output_format.q.out b/ql/src/test/results/clientpositive/binary_output_format.q.out index 46122da..ad245f2 100644 --- a/ql/src/test/results/clientpositive/binary_output_format.q.out +++ b/ql/src/test/results/clientpositive/binary_output_format.q.out @@ -113,7 +113,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -226,7 +225,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -295,7 +293,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucket1.q.out b/ql/src/test/results/clientpositive/bucket1.q.out index 163753a..738b487 100644 --- a/ql/src/test/results/clientpositive/bucket1.q.out +++ b/ql/src/test/results/clientpositive/bucket1.q.out @@ -45,7 +45,6 @@ STAGE PLANS: type: string expr: _col1 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -91,6 +90,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src + Truncated Path -> Alias: + /src [src] + Needs Tagging: false Reduce Operator Tree: Extract Select Operator @@ -125,8 +127,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /src [src] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucket2.q.out b/ql/src/test/results/clientpositive/bucket2.q.out index c889997..efd2436 100644 --- a/ql/src/test/results/clientpositive/bucket2.q.out +++ b/ql/src/test/results/clientpositive/bucket2.q.out @@ -45,7 +45,6 @@ STAGE PLANS: type: string expr: _col1 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -91,6 +90,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src + Truncated Path -> Alias: + /src [src] + Needs Tagging: false Reduce Operator Tree: Extract Select Operator @@ -125,8 +127,6 @@ STAGE PLANS: TotalFiles: 2 GatherStats: true MultiFileSpray: true - Truncated Path -> Alias: - /src [src] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucket3.q.out b/ql/src/test/results/clientpositive/bucket3.q.out index f4a997a..2825984 100644 --- a/ql/src/test/results/clientpositive/bucket3.q.out +++ b/ql/src/test/results/clientpositive/bucket3.q.out @@ -45,7 +45,6 @@ STAGE PLANS: type: string expr: _col1 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -91,6 +90,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src + Truncated Path -> Alias: + /src [src] + Needs Tagging: false Reduce Operator Tree: Extract Select Operator @@ -127,8 +129,6 @@ STAGE PLANS: TotalFiles: 2 GatherStats: true MultiFileSpray: true - Truncated Path -> Alias: - /src [src] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucket_map_join_1.q.out b/ql/src/test/results/clientpositive/bucket_map_join_1.q.out index 06a802c..0f655a8 100644 --- a/ql/src/test/results/clientpositive/bucket_map_join_1.q.out +++ b/ql/src/test/results/clientpositive/bucket_map_join_1.q.out @@ -109,7 +109,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -159,6 +158,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.table1 name: default.table1 + Truncated Path -> Alias: + /table1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -189,8 +191,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /table1 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucket_map_join_2.q.out b/ql/src/test/results/clientpositive/bucket_map_join_2.q.out index 27c741f..36af55d 100644 --- a/ql/src/test/results/clientpositive/bucket_map_join_2.q.out +++ b/ql/src/test/results/clientpositive/bucket_map_join_2.q.out @@ -109,7 +109,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -159,6 +158,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.table1 name: default.table1 + Truncated Path -> Alias: + /table1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -189,8 +191,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /table1 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_1.q.out b/ql/src/test/results/clientpositive/bucketcontext_1.q.out index 62910fb..902a42f 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_1.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_1.q.out @@ -148,7 +148,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -248,6 +247,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -278,9 +281,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -347,7 +347,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -447,6 +446,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -477,9 +480,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_2.q.out b/ql/src/test/results/clientpositive/bucketcontext_2.q.out index 0857c9d..a6100ac 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_2.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_2.q.out @@ -136,7 +136,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -236,6 +235,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -266,9 +269,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -335,7 +335,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -435,6 +434,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -465,9 +468,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_3.q.out b/ql/src/test/results/clientpositive/bucketcontext_3.q.out index 69dc2b2..a913b56 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_3.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_3.q.out @@ -136,7 +136,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -188,6 +187,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -218,8 +220,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] Stage: Stage-0 Fetch Operator @@ -286,7 +286,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -338,6 +337,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -368,8 +370,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_4.q.out b/ql/src/test/results/clientpositive/bucketcontext_4.q.out index 0d79901..efd4c20 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_4.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_4.q.out @@ -148,7 +148,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -200,6 +199,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -230,8 +232,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] Stage: Stage-0 Fetch Operator @@ -298,7 +298,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -350,6 +349,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -380,8 +382,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_5.q.out b/ql/src/test/results/clientpositive/bucketcontext_5.q.out index e5d2db4..62133f2 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_5.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_5.q.out @@ -121,7 +121,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -171,6 +170,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -201,8 +203,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big [b] Stage: Stage-0 Fetch Operator @@ -263,7 +263,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -313,6 +312,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -343,8 +345,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_6.q.out b/ql/src/test/results/clientpositive/bucketcontext_6.q.out index 99f19a0..1ad377a 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_6.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_6.q.out @@ -135,7 +135,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -235,6 +234,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -265,9 +268,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -332,7 +332,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -432,6 +431,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -462,9 +465,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_7.q.out b/ql/src/test/results/clientpositive/bucketcontext_7.q.out index 19ea4fa..ff51dc5 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_7.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_7.q.out @@ -161,7 +161,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -261,6 +260,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -291,9 +294,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -362,7 +362,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -462,6 +461,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -492,9 +495,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketcontext_8.q.out b/ql/src/test/results/clientpositive/bucketcontext_8.q.out index 9a7aaa0..02b5239 100644 --- a/ql/src/test/results/clientpositive/bucketcontext_8.q.out +++ b/ql/src/test/results/clientpositive/bucketcontext_8.q.out @@ -161,7 +161,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -261,6 +260,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -291,9 +294,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator @@ -362,7 +362,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -462,6 +461,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.bucket_big name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -492,9 +495,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketmapjoin1.q.out b/ql/src/test/results/clientpositive/bucketmapjoin1.q.out index 9f8552a..7778cc9 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin1.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin1.q.out @@ -114,7 +114,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Stage: Stage-0 Fetch Operator @@ -234,7 +233,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Stage: Stage-0 Fetch Operator @@ -430,7 +428,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -543,7 +540,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -609,7 +605,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -925,7 +920,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1051,7 +1045,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1132,7 +1125,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucketmapjoin10.q.out b/ql/src/test/results/clientpositive/bucketmapjoin10.q.out index 1a6bc06..774f354 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin10.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin10.q.out @@ -177,7 +177,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -275,6 +274,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + /srcbucket_mapjoin_part_1/part=2 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -305,9 +308,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] - /srcbucket_mapjoin_part_1/part=2 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketmapjoin11.q.out b/ql/src/test/results/clientpositive/bucketmapjoin11.q.out index bd9b1fe..c48427d 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin11.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin11.q.out @@ -191,7 +191,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -289,6 +288,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + /srcbucket_mapjoin_part_1/part=2 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -319,9 +322,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] - /srcbucket_mapjoin_part_1/part=2 [a] Stage: Stage-0 Fetch Operator @@ -432,7 +432,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -530,6 +529,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + /srcbucket_mapjoin_part_1/part=2 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -560,9 +563,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] - /srcbucket_mapjoin_part_1/part=2 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketmapjoin12.q.out b/ql/src/test/results/clientpositive/bucketmapjoin12.q.out index fc161a9..96c4880 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin12.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin12.q.out @@ -159,7 +159,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -210,6 +209,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -240,8 +242,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] Stage: Stage-0 Fetch Operator @@ -344,7 +344,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -395,6 +394,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -425,8 +427,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketmapjoin13.q.out b/ql/src/test/results/clientpositive/bucketmapjoin13.q.out index 30d8925..fed5a93 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin13.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin13.q.out @@ -157,7 +157,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -255,6 +254,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + /srcbucket_mapjoin_part_1/part=2 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -285,9 +288,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] - /srcbucket_mapjoin_part_1/part=2 [a] Stage: Stage-0 Fetch Operator @@ -412,7 +412,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -463,6 +462,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=2 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -493,8 +495,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=2 [a] Stage: Stage-0 Fetch Operator @@ -631,7 +631,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -682,6 +681,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=2 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -712,8 +714,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=2 [a] Stage: Stage-0 Fetch Operator @@ -852,7 +852,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -903,6 +902,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=2 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -933,8 +935,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=2 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketmapjoin2.q.out b/ql/src/test/results/clientpositive/bucketmapjoin2.q.out index 7f3fb3e..822de17 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin2.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin2.q.out @@ -171,7 +171,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -287,7 +286,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -353,7 +351,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -673,7 +670,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -799,7 +795,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -880,7 +875,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1369,7 +1363,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1495,7 +1488,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1576,7 +1568,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucketmapjoin3.q.out b/ql/src/test/results/clientpositive/bucketmapjoin3.q.out index 913e925..f6a582e 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin3.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin3.q.out @@ -188,7 +188,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -304,7 +303,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -370,7 +368,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -690,7 +687,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -816,7 +812,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -897,7 +892,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucketmapjoin4.q.out b/ql/src/test/results/clientpositive/bucketmapjoin4.q.out index ae0be28..30d4587 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin4.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin4.q.out @@ -188,7 +188,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -301,7 +300,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -367,7 +365,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -675,7 +672,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -798,7 +794,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -879,7 +874,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucketmapjoin5.q.out b/ql/src/test/results/clientpositive/bucketmapjoin5.q.out index 29746b4..21a47c0 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin5.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin5.q.out @@ -226,7 +226,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -390,7 +389,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -456,7 +454,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -776,7 +773,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -950,7 +946,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1031,7 +1026,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucketmapjoin8.q.out b/ql/src/test/results/clientpositive/bucketmapjoin8.q.out index 92c74a9..a68689e 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin8.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin8.q.out @@ -131,7 +131,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -182,6 +181,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -212,8 +214,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] Stage: Stage-0 Fetch Operator @@ -332,7 +332,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -383,6 +382,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -413,8 +415,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketmapjoin9.q.out b/ql/src/test/results/clientpositive/bucketmapjoin9.q.out index b7aec66..ed29dd0 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin9.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin9.q.out @@ -129,7 +129,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -180,6 +179,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -210,8 +212,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] Stage: Stage-0 Fetch Operator @@ -351,7 +351,6 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -402,6 +401,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcbucket_mapjoin_part_1 name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/part=1 [a] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -432,8 +434,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcbucket_mapjoin_part_1/part=1 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out index 1dd45d2..5c7d26a 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out @@ -146,7 +146,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -259,7 +258,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -325,7 +323,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out index 37f4a48..4329ade 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out @@ -161,7 +161,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -274,7 +273,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -340,7 +338,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out index 07dced9..301aa02 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out @@ -200,7 +200,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -350,7 +349,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -494,7 +492,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -636,7 +633,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -778,7 +774,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -920,7 +915,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1062,7 +1056,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1204,7 +1197,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1346,7 +1338,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/columnstats_partlvl.q.out b/ql/src/test/results/clientpositive/columnstats_partlvl.q.out index 9330b25..8f10b9b 100644 --- a/ql/src/test/results/clientpositive/columnstats_partlvl.q.out +++ b/ql/src/test/results/clientpositive/columnstats_partlvl.q.out @@ -126,7 +126,6 @@ STAGE PLANS: value expressions: expr: _col0 type: struct - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -177,6 +176,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.employee_part name: default.employee_part + Truncated Path -> Alias: + /employee_part/employeesalary=2000.0 [employee_part] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -207,8 +209,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /employee_part/employeesalary=2000.0 [employee_part] Stage: Stage-1 Column Stats Work @@ -333,7 +333,6 @@ STAGE PLANS: value expressions: expr: _col0 type: struct - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -384,6 +383,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.employee_part name: default.employee_part + Truncated Path -> Alias: + /employee_part/employeesalary=4000.0 [employee_part] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -414,8 +416,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /employee_part/employeesalary=4000.0 [employee_part] Stage: Stage-1 Column Stats Work diff --git a/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out b/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out index ab411fc..d8a0c33 100644 --- a/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out +++ b/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out @@ -161,7 +161,6 @@ STAGE PLANS: type: struct expr: _col2 type: struct - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -209,6 +208,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.uservisits_web_text_none name: default.uservisits_web_text_none + Truncated Path -> Alias: + /uservisits_web_text_none [uservisits_web_text_none] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -245,8 +247,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /uservisits_web_text_none [uservisits_web_text_none] Stage: Stage-1 Column Stats Work diff --git a/ql/src/test/results/clientpositive/combine2_hadoop20.q.out b/ql/src/test/results/clientpositive/combine2_hadoop20.q.out index 7edbd87..ca9da18 100644 --- a/ql/src/test/results/clientpositive/combine2_hadoop20.q.out +++ b/ql/src/test/results/clientpositive/combine2_hadoop20.q.out @@ -232,7 +232,6 @@ STAGE PLANS: value expressions: expr: _col0 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -596,6 +595,16 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.combine2 name: default.combine2 + Truncated Path -> Alias: + /combine2/value=2010-04-21 09%3A45%3A00 [combine2] + /combine2/value=val_0 [combine2] + /combine2/value=val_2 [combine2] + /combine2/value=val_4 [combine2] + /combine2/value=val_5 [combine2] + /combine2/value=val_8 [combine2] + /combine2/value=val_9 [combine2] + /combine2/value=| [combine2] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -626,15 +635,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /combine2/value=2010-04-21 09%3A45%3A00 [combine2] - /combine2/value=val_0 [combine2] - /combine2/value=val_2 [combine2] - /combine2/value=val_4 [combine2] - /combine2/value=val_5 [combine2] - /combine2/value=val_8 [combine2] - /combine2/value=val_9 [combine2] - /combine2/value=| [combine2] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/ctas_hadoop20.q.out b/ql/src/test/results/clientpositive/ctas_hadoop20.q.out index ead9ea5..48bf035 100644 --- a/ql/src/test/results/clientpositive/ctas_hadoop20.q.out +++ b/ql/src/test/results/clientpositive/ctas_hadoop20.q.out @@ -763,7 +763,6 @@ STAGE PLANS: type: string expr: _col1 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -809,6 +808,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src + Truncated Path -> Alias: + /src [src] + Needs Tagging: false Reduce Operator Tree: Extract Limit @@ -827,8 +829,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /src [src] Stage: Stage-2 Map Reduce @@ -847,7 +847,6 @@ STAGE PLANS: type: string expr: _col1 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -867,6 +866,9 @@ STAGE PLANS: columns _col0,_col1 columns.types string,string escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Extract Limit @@ -891,8 +893,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/filter_join_breaktask.q.out b/ql/src/test/results/clientpositive/filter_join_breaktask.q.out index de80d6e..9560d20 100644 --- a/ql/src/test/results/clientpositive/filter_join_breaktask.q.out +++ b/ql/src/test/results/clientpositive/filter_join_breaktask.q.out @@ -81,7 +81,6 @@ STAGE PLANS: value expressions: expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -130,6 +129,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.filter_join_breaktask name: default.filter_join_breaktask + Truncated Path -> Alias: + /filter_join_breaktask/ds=2008-04-08 [f, m] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -154,8 +156,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /filter_join_breaktask/ds=2008-04-08 [f, m] Stage: Stage-2 Map Reduce @@ -194,7 +194,6 @@ STAGE PLANS: value expressions: expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -259,6 +258,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.filter_join_breaktask name: default.filter_join_breaktask + Truncated Path -> Alias: + /filter_join_breaktask/ds=2008-04-08 [g] +#### A masked pattern was here #### + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -293,9 +296,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /filter_join_breaktask/ds=2008-04-08 [g] -#### A masked pattern was here #### Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out index 4058cbb..317ce80 100644 --- a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out +++ b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out @@ -68,7 +68,6 @@ STAGE PLANS: type: bigint expr: _col3 type: double - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -164,6 +163,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [src] + /srcpart/ds=2008-04-08/hr=12 [src] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -208,9 +211,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [src] - /srcpart/ds=2008-04-08/hr=12 [src] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out index 5402136..07d4d52 100644 --- a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out +++ b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out @@ -78,7 +78,6 @@ STAGE PLANS: type: double expr: _col6 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -174,6 +173,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [src] + /srcpart/ds=2008-04-08/hr=12 [src] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -224,9 +227,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [src] - /srcpart/ds=2008-04-08/hr=12 [src] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/groupby_ppr.q.out b/ql/src/test/results/clientpositive/groupby_ppr.q.out index 0b1fb51..7fc1a00 100644 --- a/ql/src/test/results/clientpositive/groupby_ppr.q.out +++ b/ql/src/test/results/clientpositive/groupby_ppr.q.out @@ -51,7 +51,6 @@ STAGE PLANS: expr: substr(key, 1, 1) type: string tag: -1 - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -147,6 +146,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [src] + /srcpart/ds=2008-04-08/hr=12 [src] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -191,9 +194,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [src] - /srcpart/ds=2008-04-08/hr=12 [src] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out index e4fa6c5..ccabf06 100644 --- a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out +++ b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out @@ -53,7 +53,6 @@ STAGE PLANS: expr: substr(key, 1, 1) type: string tag: -1 - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -149,6 +148,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [src] + /srcpart/ds=2008-04-08/hr=12 [src] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -199,9 +202,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [src] - /srcpart/ds=2008-04-08/hr=12 [src] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/groupby_sort_1.q.out b/ql/src/test/results/clientpositive/groupby_sort_1.q.out index 965bb00..699c743 100644 --- a/ql/src/test/results/clientpositive/groupby_sort_1.q.out +++ b/ql/src/test/results/clientpositive/groupby_sort_1.q.out @@ -112,7 +112,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -227,7 +226,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -293,7 +291,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -443,7 +440,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -493,6 +489,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -538,8 +537,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t1 [t1] Stage: Stage-0 Move Operator @@ -694,7 +691,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -819,7 +815,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -900,7 +895,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1087,7 +1081,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1212,7 +1205,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1293,7 +1285,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1503,7 +1494,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1618,7 +1608,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1684,7 +1673,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1880,7 +1868,6 @@ STAGE PLANS: value expressions: expr: _col3 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1930,6 +1917,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -1979,8 +1969,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t1 [t1] Stage: Stage-0 Move Operator @@ -2145,7 +2133,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2195,6 +2182,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2245,8 +2235,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t1 [t1] Stage: Stage-0 Move Operator @@ -2444,7 +2432,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2494,6 +2481,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq1:t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2540,8 +2530,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq1:t1] Stage: Stage-0 Move Operator @@ -2830,7 +2818,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2955,7 +2942,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3036,7 +3022,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3269,7 +3254,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3319,6 +3303,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [null-subquery2:subq1-subquery2:t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -3351,8 +3338,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [null-subquery2:subq1-subquery2:t1] Stage: Stage-2 Map Reduce @@ -3459,7 +3444,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3601,7 +3585,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3682,7 +3665,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3961,7 +3943,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4011,6 +3992,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq1:t1, subq2:t1] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -4056,8 +4040,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq1:t1, subq2:t1] Stage: Stage-0 Move Operator @@ -4275,7 +4257,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4325,6 +4306,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq2:t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -4361,8 +4345,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq2:t1] Stage: Stage-1 Map Reduce @@ -4423,7 +4405,6 @@ STAGE PLANS: type: string expr: _col1 type: bigint - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4489,6 +4470,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq1:t1] +#### A masked pattern was here #### + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -4529,9 +4514,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq1:t1] -#### A masked pattern was here #### Stage: Stage-0 Fetch Operator @@ -4698,7 +4680,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4748,6 +4729,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t2 name: default.t2 + Truncated Path -> Alias: + /t2 [t2] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -4794,8 +4778,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t2 [t2] Stage: Stage-0 Move Operator @@ -5044,7 +5026,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5169,7 +5150,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5250,7 +5230,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5575,7 +5554,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5690,7 +5668,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5756,7 +5733,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6051,7 +6027,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6176,7 +6151,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6257,7 +6231,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6582,7 +6555,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6707,7 +6679,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6788,7 +6759,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/groupby_sort_6.q.out b/ql/src/test/results/clientpositive/groupby_sort_6.q.out index 49ea4ac..a4e8707 100644 --- a/ql/src/test/results/clientpositive/groupby_sort_6.q.out +++ b/ql/src/test/results/clientpositive/groupby_sort_6.q.out @@ -389,7 +389,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -438,6 +437,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1/ds=2 [t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -484,8 +486,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t1/ds=2 [t1] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/groupby_sort_skew_1.q.out b/ql/src/test/results/clientpositive/groupby_sort_skew_1.q.out index 10536b2..a2df6c4 100644 --- a/ql/src/test/results/clientpositive/groupby_sort_skew_1.q.out +++ b/ql/src/test/results/clientpositive/groupby_sort_skew_1.q.out @@ -112,7 +112,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -227,7 +226,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -293,7 +291,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -442,7 +439,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -492,6 +488,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -519,8 +518,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [t1] Stage: Stage-2 Map Reduce @@ -542,7 +539,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -562,6 +558,9 @@ STAGE PLANS: columns _col0,_col1,_col2 columns.types string,string,bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -607,8 +606,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Move Operator @@ -763,7 +760,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -888,7 +884,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -969,7 +964,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1156,7 +1150,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1281,7 +1274,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1362,7 +1354,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1572,7 +1563,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1687,7 +1677,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1753,7 +1742,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1946,7 +1934,6 @@ STAGE PLANS: value expressions: expr: _col3 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1996,6 +1983,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2025,8 +2015,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [t1] Stage: Stage-2 Map Reduce @@ -2052,7 +2040,6 @@ STAGE PLANS: value expressions: expr: _col3 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2072,6 +2059,9 @@ STAGE PLANS: columns _col0,_col1,_col2,_col3 columns.types string,int,string,bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2121,8 +2111,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Move Operator @@ -2286,7 +2274,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2336,6 +2323,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2363,8 +2353,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [t1] Stage: Stage-2 Map Reduce @@ -2386,7 +2374,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2406,6 +2393,9 @@ STAGE PLANS: columns _col0,_col1,_col2 columns.types string,double,bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2456,8 +2446,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Move Operator @@ -2656,7 +2644,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2706,6 +2693,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq1:t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2731,8 +2721,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq1:t1] Stage: Stage-2 Map Reduce @@ -2750,7 +2738,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2770,6 +2757,9 @@ STAGE PLANS: columns _col0,_col1 columns.types double,bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -2816,8 +2806,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Move Operator @@ -3106,7 +3094,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3231,7 +3218,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3312,7 +3298,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3546,7 +3531,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3596,6 +3580,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [null-subquery2:subq1-subquery2:t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -3621,8 +3608,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [null-subquery2:subq1-subquery2:t1] Stage: Stage-10 Map Reduce @@ -3640,7 +3625,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3660,6 +3644,9 @@ STAGE PLANS: columns _col0,_col1 columns.types double,bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -3692,8 +3679,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-2 Map Reduce @@ -3800,7 +3785,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -3942,7 +3926,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4023,7 +4006,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4302,7 +4284,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4352,6 +4333,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq1:t1, subq2:t1] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -4397,8 +4381,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq1:t1, subq2:t1] Stage: Stage-0 Move Operator @@ -4615,7 +4597,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4665,6 +4646,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq2:t1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -4692,8 +4676,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq2:t1] Stage: Stage-3 Map Reduce @@ -4715,7 +4697,6 @@ STAGE PLANS: value expressions: expr: _col2 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4735,6 +4716,9 @@ STAGE PLANS: columns _col0,_col1,_col2 columns.types string,string,bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -4771,8 +4755,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-1 Map Reduce @@ -4833,7 +4815,6 @@ STAGE PLANS: type: string expr: _col1 type: bigint - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4899,6 +4880,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 + Truncated Path -> Alias: + /t1 [subq1:t1] +#### A masked pattern was here #### + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -4939,9 +4924,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t1 [subq1:t1] -#### A masked pattern was here #### Stage: Stage-0 Fetch Operator @@ -5109,7 +5091,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5159,6 +5140,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t2 name: default.t2 + Truncated Path -> Alias: + /t2 [t2] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -5184,8 +5168,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /t2 [t2] Stage: Stage-2 Map Reduce @@ -5203,7 +5185,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5223,6 +5204,9 @@ STAGE PLANS: columns _col0,_col1 columns.types string,bigint escape.delim \ + Truncated Path -> Alias: +#### A masked pattern was here #### + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -5269,8 +5253,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: -#### A masked pattern was here #### Stage: Stage-0 Move Operator @@ -5519,7 +5501,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5644,7 +5625,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5725,7 +5705,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6050,7 +6029,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6165,7 +6143,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6231,7 +6208,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6526,7 +6502,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6651,7 +6626,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -6732,7 +6706,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -7057,7 +7030,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -7182,7 +7154,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -7263,7 +7234,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/input23.q.out b/ql/src/test/results/clientpositive/input23.q.out index 76b9a07..9f897d0 100644 --- a/ql/src/test/results/clientpositive/input23.q.out +++ b/ql/src/test/results/clientpositive/input23.q.out @@ -52,7 +52,6 @@ STAGE PLANS: type: string expr: hr type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -102,6 +101,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -149,8 +151,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/input4.q.out b/ql/src/test/results/clientpositive/input4.q.out index 1485e4c..8717143 100644 --- a/ql/src/test/results/clientpositive/input4.q.out +++ b/ql/src/test/results/clientpositive/input4.q.out @@ -48,7 +48,7 @@ PREHOOK: type: QUERY POSTHOOK: query: EXPLAIN FORMATTED SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias POSTHOOK: type: QUERY -{"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Split Sample:":{},"Alias -> Map Operator Tree:":{"input4alias":{"TS_0":{"SEL_1":{"FS_2":{"File Output Operator":{"GlobalTableId:":"0","compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1"}}},"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"ROOT STAGE":"TRUE"}},"ABSTRACT SYNTAX TREE":"(TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME INPUT4) Input4Alias)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL Input4Alias) VALUE)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL Input4Alias) KEY)))))"} +{"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map:":{"Split Sample:":{},"Alias -> Map Operator Tree:":{"input4alias":{"TS_0":{"SEL_1":{"FS_2":{"File Output Operator":{"GlobalTableId:":"0","compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}}}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1"}}},"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"ROOT STAGE":"TRUE"}},"ABSTRACT SYNTAX TREE":"(TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME INPUT4) Input4Alias)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL Input4Alias) VALUE)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL Input4Alias) KEY)))))"} PREHOOK: query: SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias PREHOOK: type: QUERY PREHOOK: Input: default@input4 diff --git a/ql/src/test/results/clientpositive/input42.q.out b/ql/src/test/results/clientpositive/input42.q.out index 2ccc32c..8ebb15e 100644 --- a/ql/src/test/results/clientpositive/input42.q.out +++ b/ql/src/test/results/clientpositive/input42.q.out @@ -47,7 +47,6 @@ STAGE PLANS: type: string expr: _col3 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -143,6 +142,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [a] + /srcpart/ds=2008-04-08/hr=12 [a] + Needs Tagging: false Reduce Operator Tree: Extract File Output Operator @@ -163,9 +166,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [a] - /srcpart/ds=2008-04-08/hr=12 [a] Stage: Stage-0 Fetch Operator @@ -1238,7 +1238,6 @@ STAGE PLANS: type: string expr: _col3 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1334,6 +1333,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [a] + /srcpart/ds=2008-04-08/hr=12 [a] + Needs Tagging: false Reduce Operator Tree: Extract File Output Operator @@ -1354,9 +1357,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [a] - /srcpart/ds=2008-04-08/hr=12 [a] Stage: Stage-0 Fetch Operator @@ -1807,7 +1807,6 @@ STAGE PLANS: type: string expr: _col3 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1903,6 +1902,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [a] + /srcpart/ds=2008-04-08/hr=12 [a] + Needs Tagging: false Reduce Operator Tree: Extract File Output Operator @@ -1923,9 +1926,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [a] - /srcpart/ds=2008-04-08/hr=12 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/input_part1.q.out b/ql/src/test/results/clientpositive/input_part1.q.out index a8222d4..f595c8a 100644 --- a/ql/src/test/results/clientpositive/input_part1.q.out +++ b/ql/src/test/results/clientpositive/input_part1.q.out @@ -72,7 +72,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -187,7 +186,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -253,7 +251,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/input_part2.q.out b/ql/src/test/results/clientpositive/input_part2.q.out index 7f8dd01..07fb320 100644 --- a/ql/src/test/results/clientpositive/input_part2.q.out +++ b/ql/src/test/results/clientpositive/input_part2.q.out @@ -126,7 +126,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -288,7 +287,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -354,7 +352,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -461,7 +458,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -527,7 +523,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/input_part7.q.out b/ql/src/test/results/clientpositive/input_part7.q.out index 300173d..7a69988 100644 --- a/ql/src/test/results/clientpositive/input_part7.q.out +++ b/ql/src/test/results/clientpositive/input_part7.q.out @@ -131,7 +131,6 @@ STAGE PLANS: type: string expr: _col3 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -227,6 +226,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y] + /srcpart/ds=2008-04-08/hr=12 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y] + Needs Tagging: false Reduce Operator Tree: Extract File Output Operator @@ -247,9 +250,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y] - /srcpart/ds=2008-04-08/hr=12 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/input_part9.q.out b/ql/src/test/results/clientpositive/input_part9.q.out index e343c9c..a37a355 100644 --- a/ql/src/test/results/clientpositive/input_part9.q.out +++ b/ql/src/test/results/clientpositive/input_part9.q.out @@ -52,7 +52,6 @@ STAGE PLANS: type: string expr: _col3 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -148,6 +147,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [x] + /srcpart/ds=2008-04-08/hr=12 [x] + Needs Tagging: false Reduce Operator Tree: Extract File Output Operator @@ -168,9 +171,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [x] - /srcpart/ds=2008-04-08/hr=12 [x] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/join17.q.out b/ql/src/test/results/clientpositive/join17.q.out index a20acaf..f5c626d 100644 --- a/ql/src/test/results/clientpositive/join17.q.out +++ b/ql/src/test/results/clientpositive/join17.q.out @@ -59,7 +59,6 @@ STAGE PLANS: type: string expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -105,6 +104,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src + Truncated Path -> Alias: + /src [src2, src1] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -149,8 +151,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /src [src2, src1] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/join26.q.out b/ql/src/test/results/clientpositive/join26.q.out index ecdbda6..4c4977a 100644 --- a/ql/src/test/results/clientpositive/join26.q.out +++ b/ql/src/test/results/clientpositive/join26.q.out @@ -128,7 +128,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -243,7 +242,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -309,7 +307,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/join32.q.out b/ql/src/test/results/clientpositive/join32.q.out index 92d81b9..435650d 100644 --- a/ql/src/test/results/clientpositive/join32.q.out +++ b/ql/src/test/results/clientpositive/join32.q.out @@ -132,7 +132,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/join32_lessSize.q.out b/ql/src/test/results/clientpositive/join32_lessSize.q.out index 82b3e4a..cefeb4b 100644 --- a/ql/src/test/results/clientpositive/join32_lessSize.q.out +++ b/ql/src/test/results/clientpositive/join32_lessSize.q.out @@ -93,7 +93,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -256,7 +255,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -563,7 +561,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -757,7 +754,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -995,7 +991,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1202,7 +1197,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1357,7 +1351,6 @@ STAGE PLANS: value expressions: expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1461,6 +1454,11 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src1 name: default.src1 + Truncated Path -> Alias: + /src [y] + /src1 [z] +#### A masked pattern was here #### + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -1510,10 +1508,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /src [y] - /src1 [z] -#### A masked pattern was here #### PREHOOK: query: INSERT OVERWRITE TABLE dest_j1 @@ -1733,7 +1727,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1896,7 +1889,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2220,7 +2212,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2388,7 +2379,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/join33.q.out b/ql/src/test/results/clientpositive/join33.q.out index 92d81b9..435650d 100644 --- a/ql/src/test/results/clientpositive/join33.q.out +++ b/ql/src/test/results/clientpositive/join33.q.out @@ -132,7 +132,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/join34.q.out b/ql/src/test/results/clientpositive/join34.q.out index c108931..c9d42f0 100644 --- a/ql/src/test/results/clientpositive/join34.q.out +++ b/ql/src/test/results/clientpositive/join34.q.out @@ -187,7 +187,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/join35.q.out b/ql/src/test/results/clientpositive/join35.q.out index 7074628..dc1faca 100644 --- a/ql/src/test/results/clientpositive/join35.q.out +++ b/ql/src/test/results/clientpositive/join35.q.out @@ -81,7 +81,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -127,6 +126,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src + Truncated Path -> Alias: + /src [null-subquery1:subq1-subquery1:x] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -159,8 +161,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /src [null-subquery1:subq1-subquery1:x] Stage: Stage-8 Conditional Operator @@ -289,7 +289,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -487,7 +486,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -621,7 +619,6 @@ STAGE PLANS: type: string expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -699,6 +696,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src1 name: default.src1 + Truncated Path -> Alias: + /src1 [x] +#### A masked pattern was here #### + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -741,9 +742,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /src1 [x] -#### A masked pattern was here #### Stage: Stage-4 Map Reduce @@ -783,7 +781,6 @@ STAGE PLANS: value expressions: expr: _col1 type: bigint - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -829,6 +826,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src + Truncated Path -> Alias: + /src [null-subquery2:subq1-subquery2:x1] + Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: @@ -861,8 +861,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /src [null-subquery2:subq1-subquery2:x1] PREHOOK: query: INSERT OVERWRITE TABLE dest_j1 diff --git a/ql/src/test/results/clientpositive/join9.q.out b/ql/src/test/results/clientpositive/join9.q.out index 7569e6f..4c40182 100644 --- a/ql/src/test/results/clientpositive/join9.q.out +++ b/ql/src/test/results/clientpositive/join9.q.out @@ -55,7 +55,6 @@ STAGE PLANS: value expressions: expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -147,6 +146,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /src [src2] + /srcpart/ds=2008-04-08/hr=12 [src1] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -187,9 +190,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /src [src2] - /srcpart/ds=2008-04-08/hr=12 [src1] Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/join_filters_overlap.q.out b/ql/src/test/results/clientpositive/join_filters_overlap.q.out index 0adccba..ca1b6a0 100644 --- a/ql/src/test/results/clientpositive/join_filters_overlap.q.out +++ b/ql/src/test/results/clientpositive/join_filters_overlap.q.out @@ -90,7 +90,6 @@ STAGE PLANS: type: int expr: value type: int - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -136,6 +135,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.a name: default.a + Truncated Path -> Alias: + /a [b, c, a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -186,8 +188,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /a [b, c, a] Stage: Stage-0 Fetch Operator @@ -297,7 +297,6 @@ STAGE PLANS: type: int expr: value type: int - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -343,6 +342,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.a name: default.a + Truncated Path -> Alias: + /a [b, c, a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -393,8 +395,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /a [b, c, a] Stage: Stage-0 Fetch Operator @@ -504,7 +504,6 @@ STAGE PLANS: type: int expr: value type: int - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -550,6 +549,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.a name: default.a + Truncated Path -> Alias: + /a [b, c, a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -600,8 +602,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /a [b, c, a] Stage: Stage-0 Fetch Operator @@ -729,7 +729,6 @@ STAGE PLANS: type: int expr: value type: int - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -775,6 +774,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.a name: default.a + Truncated Path -> Alias: + /a [d, b, c, a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -833,8 +835,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /a [d, b, c, a] Stage: Stage-0 Fetch Operator @@ -958,7 +958,6 @@ STAGE PLANS: type: int expr: value type: int - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1004,6 +1003,9 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.a name: default.a + Truncated Path -> Alias: + /a [d, b, c, a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -1061,8 +1063,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /a [d, b, c, a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/join_map_ppr.q.out b/ql/src/test/results/clientpositive/join_map_ppr.q.out index 483239c..ef6cd5a 100644 --- a/ql/src/test/results/clientpositive/join_map_ppr.q.out +++ b/ql/src/test/results/clientpositive/join_map_ppr.q.out @@ -130,7 +130,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -245,7 +244,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -311,7 +309,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -683,7 +680,6 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -808,7 +804,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -889,7 +884,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/load_dyn_part8.q.out b/ql/src/test/results/clientpositive/load_dyn_part8.q.out index 8b1e627..69098f7 100644 --- a/ql/src/test/results/clientpositive/load_dyn_part8.q.out +++ b/ql/src/test/results/clientpositive/load_dyn_part8.q.out @@ -136,7 +136,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/louter_join_ppr.q.out b/ql/src/test/results/clientpositive/louter_join_ppr.q.out index e8a5def..fe80596 100644 --- a/ql/src/test/results/clientpositive/louter_join_ppr.q.out +++ b/ql/src/test/results/clientpositive/louter_join_ppr.q.out @@ -73,7 +73,6 @@ STAGE PLANS: type: string expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -211,6 +210,11 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /src [a] + /srcpart/ds=2008-04-08/hr=11 [b] + /srcpart/ds=2008-04-08/hr=12 [b] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -254,10 +258,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /src [a] - /srcpart/ds=2008-04-08/hr=11 [b] - /srcpart/ds=2008-04-08/hr=12 [b] Stage: Stage-0 Fetch Operator @@ -379,7 +379,6 @@ STAGE PLANS: type: string expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -609,6 +608,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /src [b] + /srcpart/ds=2008-04-08/hr=11 [a] + /srcpart/ds=2008-04-08/hr=12 [a] + /srcpart/ds=2008-04-09/hr=11 [a] + /srcpart/ds=2008-04-09/hr=12 [a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -657,12 +663,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /src [b] - /srcpart/ds=2008-04-08/hr=11 [a] - /srcpart/ds=2008-04-08/hr=12 [a] - /srcpart/ds=2008-04-09/hr=11 [a] - /srcpart/ds=2008-04-09/hr=12 [a] Stage: Stage-0 Fetch Operator @@ -788,7 +788,6 @@ STAGE PLANS: type: string expr: ds type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1018,6 +1017,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /src [a] + /srcpart/ds=2008-04-08/hr=11 [b] + /srcpart/ds=2008-04-08/hr=12 [b] + /srcpart/ds=2008-04-09/hr=11 [b] + /srcpart/ds=2008-04-09/hr=12 [b] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -1061,12 +1067,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /src [a] - /srcpart/ds=2008-04-08/hr=11 [b] - /srcpart/ds=2008-04-08/hr=12 [b] - /srcpart/ds=2008-04-09/hr=11 [b] - /srcpart/ds=2008-04-09/hr=12 [b] Stage: Stage-0 Fetch Operator @@ -1190,7 +1190,6 @@ STAGE PLANS: type: string expr: value type: string - Needs Tagging: true Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1328,6 +1327,11 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart + Truncated Path -> Alias: + /src [b] + /srcpart/ds=2008-04-08/hr=11 [a] + /srcpart/ds=2008-04-08/hr=12 [a] + Needs Tagging: true Reduce Operator Tree: Join Operator condition map: @@ -1371,10 +1375,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Truncated Path -> Alias: - /src [b] - /srcpart/ds=2008-04-08/hr=11 [a] - /srcpart/ds=2008-04-08/hr=12 [a] Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/macro.q.out b/ql/src/test/results/clientpositive/macro.q.out index ce2658c..0901a3e 100644 --- a/ql/src/test/results/clientpositive/macro.q.out +++ b/ql/src/test/results/clientpositive/macro.q.out @@ -90,7 +90,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -240,7 +239,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -411,7 +409,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git a/ql/src/test/results/clientpositive/merge3.q.out b/ql/src/test/results/clientpositive/merge3.q.out index d3c700b..244ec7b 100644 --- a/ql/src/test/results/clientpositive/merge3.q.out +++ b/ql/src/test/results/clientpositive/merge3.q.out @@ -100,7 +100,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -200,7 +199,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -248,7 +246,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2423,7 +2420,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2587,7 +2583,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2656,7 +2651,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4853,7 +4847,6 @@ STAGE PLANS: type: string expr: _col2 type: string - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -4947,6 +4940,10 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.merge_src_part name: default.merge_src_part + Truncated Path -> Alias: + /merge_src_part/ds=2008-04-08 [s:merge_src_part] + /merge_src_part/ds=2008-04-09 [s:merge_src_part] + Needs Tagging: false Reduce Operator Tree: Extract Select Operator @@ -4983,9 +4980,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: true MultiFileSpray: false - Truncated Path -> Alias: - /merge_src_part/ds=2008-04-08 [s:merge_src_part] - /merge_src_part/ds=2008-04-09 [s:merge_src_part] Stage: Stage-7 Conditional Operator @@ -5053,7 +5047,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -5122,7 +5115,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: