diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index c777e45..c7aec5e 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -542,6 +542,8 @@ spark.query.files=add_part_multiple.q, \ groupby_complex_types.q, \ groupby_complex_types_multi_single_reducer.q, \ groupby_cube1.q, \ + groupby_map_ppr.q, \ + groupby_map_ppr_multi_distinct.q, \ groupby_multi_insert_common_distinct.q, \ groupby_multi_single_reducer.q, \ groupby_multi_single_reducer2.q, \ diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java index 6a7733c..39d8c92 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java @@ -489,6 +489,7 @@ public Object process(ReduceSinkOperator cRS, ReduceSinkDeduplicateProcCtx dedup if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { CorrelationUtilities.replaceReduceSinkWithSelectOperator( cRS, dedupCtx.getPctx(), dedupCtx); + pRS.getConf().setDeduplicated(true); return true; } return false; @@ -511,6 +512,7 @@ public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { CorrelationUtilities.removeReduceSinkForGroupBy( cRS, cGBY, dedupCtx.getPctx(), dedupCtx); + pRS.getConf().setDeduplicated(true); return true; } return false; @@ -529,6 +531,12 @@ public Object process(ReduceSinkOperator cRS, ReduceSinkDeduplicateProcCtx dedup pJoin.getConf().setFixedAsSorted(true); CorrelationUtilities.replaceReduceSinkWithSelectOperator( cRS, dedupCtx.getPctx(), dedupCtx); + ReduceSinkOperator pRS = + CorrelationUtilities.findPossibleParent( + pJoin, ReduceSinkOperator.class, dedupCtx.trustScript()); + if (pRS != null) { + pRS.getConf().setDeduplicated(true); + } return true; } return false; @@ -547,6 +555,12 @@ public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, pJoin.getConf().setFixedAsSorted(true); CorrelationUtilities.removeReduceSinkForGroupBy( cRS, cGBY, dedupCtx.getPctx(), dedupCtx); + ReduceSinkOperator pRS = + CorrelationUtilities.findPossibleParent( + pJoin, ReduceSinkOperator.class, dedupCtx.trustScript()); + if (pRS != null) { + pRS.getConf().setDeduplicated(true); + } return true; } return false; @@ -565,6 +579,7 @@ public Object process(ReduceSinkOperator cRS, ReduceSinkDeduplicateProcCtx dedup if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { CorrelationUtilities.replaceReduceSinkWithSelectOperator( cRS, dedupCtx.getPctx(), dedupCtx); + pRS.getConf().setDeduplicated(true); return true; } return false; @@ -581,6 +596,7 @@ public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, start, ReduceSinkOperator.class, dedupCtx.trustScript()); if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { CorrelationUtilities.removeReduceSinkForGroupBy(cRS, cGBY, dedupCtx.getPctx(), dedupCtx); + pRS.getConf().setDeduplicated(true); return true; } return false; diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java index 92600be..b3d39ce 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java @@ -131,7 +131,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procContext, edgeType = EdgeType.CUSTOM_SIMPLE_EDGE; } }*/ - SparkEdgeProperty edgeProp = new SparkEdgeProperty(0/*null, edgeType, numBuckets*/); + SparkEdgeProperty edgeProp = new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_NONE); if (mapJoinWork != null) { for (BaseWork myWork: mapJoinWork) { diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index e8e18a7..5394e67 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -40,13 +40,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.UnionWork; +import org.apache.hadoop.hive.ql.plan.*; import java.util.ArrayList; import java.util.Deque; @@ -117,13 +111,19 @@ public ReduceWork createReduceWork(GenSparkProcContext context, Operator root sparkWork.add(reduceWork); - // Use group-by as the default shuffler - SparkEdgeProperty edgeProp = new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_GROUP, - reduceWork.getNumReduceTasks()); - + SparkEdgeProperty edgeProp = new SparkEdgeProperty(); + edgeProp.setNumPartitions(reduceWork.getNumReduceTasks()); String sortOrder = Strings.nullToEmpty(reduceSink.getConf().getOrder()).trim(); - if (!sortOrder.isEmpty() && isSortNecessary(reduceSink)) { - edgeProp.setShuffleSort(); + + // test if we need group-by shuffle + if (reduceSink.getChildOperators().size() == 1 && + reduceSink.getChildOperators().get(0) instanceof GroupByOperator) { + edgeProp.setShuffleGroup(); + // test if the group by needs partition level sort, if so, use the MR style shuffle + // SHUFFLE_SORT shouldn't be used for this purpose, see HIVE-8542 + if (!sortOrder.isEmpty() && GBYNeedParLevelOrder(reduceSink)) { + edgeProp.setMRShuffle(); + } } if (reduceWork.getReducer() instanceof JoinOperator) { @@ -131,6 +131,25 @@ public ReduceWork createReduceWork(GenSparkProcContext context, Operator root edgeProp.setMRShuffle(); } + // test if we need total order, if so, we can either use MR shuffle + set #reducer to 1, + // or we can use SHUFFLE_SORT + if (!edgeProp.isShuffleGroup() && !edgeProp.isMRShuffle() && !sortOrder.isEmpty()) { + if (reduceSink.getConf().getPartitionCols() == null || + reduceSink.getConf().getPartitionCols().isEmpty() || + isSameENDList(reduceSink.getConf().getPartitionCols(), + reduceSink.getConf().getKeyCols())) { + edgeProp.setShuffleSort(); + } else { + edgeProp.setMRShuffle(); + } + } + + // set to groupby-shuffle if it's still NONE + // simple distribute-by goes here + if (edgeProp.isShuffleNone()) { + edgeProp.setShuffleGroup(); + } + //If its a FileSink to bucketed files, also use MR-style shuffle to get compatible taskId for bucket-name FileSinkOperator fso = getChildOperator(reduceWork.getReducer(), FileSinkOperator.class); if (fso != null) { @@ -316,19 +335,23 @@ public void processFileSink(GenSparkProcContext context, FileSinkOperator fileSi } /** - * Test if the sort order in the RS is necessary. - * Unnecessary sort is mainly introduced when GBY is created. Therefore, if the sorting + * Test if we need partition level order for group by query. + * GBY needs partition level order when distinct is present. Therefore, if the sorting * keys, partitioning keys and grouping keys are the same, we ignore the sort and use * GroupByShuffler to shuffle the data. In this case a group-by transformation should be * sufficient to produce the correct results, i.e. data is properly grouped by the keys * but keys are not guaranteed to be sorted. */ - public static boolean isSortNecessary(ReduceSinkOperator reduceSinkOperator) { + public static boolean GBYNeedParLevelOrder(ReduceSinkOperator reduceSinkOperator) { + // if the RS is deduplicated, enforce sorting anyway + if (reduceSinkOperator.getConf().isDeduplicated()) { + return true; + } List> children = reduceSinkOperator.getChildOperators(); if (children != null && children.size() == 1 && children.get(0) instanceof GroupByOperator) { GroupByOperator child = (GroupByOperator) children.get(0); - if (reduceSinkOperator.getConf().getKeyCols().equals( + if (isSameENDList(reduceSinkOperator.getConf().getKeyCols(), reduceSinkOperator.getConf().getPartitionCols()) && reduceSinkOperator.getConf().getKeyCols().size() == child.getConf().getKeys().size()) { return false; @@ -337,6 +360,27 @@ public static boolean isSortNecessary(ReduceSinkOperator reduceSinkOperator) { return true; } + /** + * Test if two lists of ExprNodeDesc are semantically same + */ + public static boolean isSameENDList(List list1, List list2) { + if (list1 != list2) { + if (list1 != null && list2 != null) { + if (list1.size() != list2.size()) { + return false; + } + for (int i = 0; i < list1.size(); i++) { + if (!list1.get(i).isSame(list2.get(i))) { + return false; + } + } + } else { + return false; + } + } + return true; + } + public static T getChildOperator(Operator op, Class klazz) throws SemanticException { if (klazz.isInstance(op)) { return (T) op; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java index 7de79b6..39a33cc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java @@ -24,15 +24,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.ql.exec.DummyStoreOperator; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorFactory; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; +import org.apache.hadoop.hive.ql.exec.*; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; @@ -311,17 +303,40 @@ public Object process(Node nd, Stack stack, if (!context.connectedReduceSinks.contains(rs)) { // add dependency between the two work items - // Use group-by as the default shuffler - SparkEdgeProperty edgeProp = new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_GROUP, - rs.getConf().getNumReducers()); + SparkEdgeProperty edgeProp = new SparkEdgeProperty(); + edgeProp.setNumPartitions(rs.getConf().getNumReducers()); String sortOrder = Strings.nullToEmpty(rs.getConf().getOrder()).trim(); - if (!sortOrder.isEmpty() && GenSparkUtils.isSortNecessary(rs)) { - edgeProp.setShuffleSort(); + // test if we need group-by shuffle + if (rs.getChildOperators().size() == 1 && + rs.getChildOperators().get(0) instanceof GroupByOperator) { + edgeProp.setShuffleGroup(); + // test if the group by needs partition level sort, if so, use the MR style shuffle + // SHUFFLE_SORT shouldn't be used for this purpose, see HIVE-8542 + if (!sortOrder.isEmpty() && GenSparkUtils.GBYNeedParLevelOrder(rs)) { + edgeProp.setMRShuffle(); + } } if (rWork.getReducer() instanceof JoinOperator) { //reduce-side join, use MR-style shuffle edgeProp.setMRShuffle(); } + // test if we need total order, if so, we can either use MR shuffle + set #reducer to 1, + // or we can use SHUFFLE_SORT + if (!edgeProp.isShuffleGroup() && !edgeProp.isMRShuffle() && !sortOrder.isEmpty()) { + if (rs.getConf().getPartitionCols() == null || + rs.getConf().getPartitionCols().isEmpty() || + GenSparkUtils.isSameENDList(rs.getConf().getPartitionCols(), + rs.getConf().getKeyCols())) { + edgeProp.setShuffleSort(); + } else { + edgeProp.setMRShuffle(); + } + } + // set to groupby-shuffle if it's still NONE + // simple distribute-by goes here + if (edgeProp.isShuffleNone()) { + edgeProp.setShuffleGroup(); + } //If its a FileSink to bucketed files, also use MR-style shuffle to get compatible taskId for bucket-name FileSinkOperator fso = GenSparkUtils.getChildOperator(rWork.getReducer(), FileSinkOperator.class); if (fso != null) { diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java index 57beb69..c8aafd6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java @@ -111,6 +111,9 @@ private ReducerTraits(int trait) { // Write type, since this needs to calculate buckets differently for updates and deletes private AcidUtils.Operation writeType; + // used for spark, remember if the RS is deduplicated, if so, spark will enforce sorting + private transient boolean isDeduplicated = false; + private static transient Log LOG = LogFactory.getLog(ReduceSinkDesc.class); public ReduceSinkDesc() { } @@ -165,6 +168,7 @@ public Object clone() { desc.setStatistics(this.getStatistics()); desc.setSkipTag(skipTag); desc.reduceTraits = reduceTraits.clone(); + desc.setDeduplicated(isDeduplicated); return desc; } @@ -407,4 +411,12 @@ public final void setReducerTraits(EnumSet traits) { public AcidUtils.Operation getWriteType() { return writeType; } + + public boolean isDeduplicated() { + return isDeduplicated; + } + + public void setDeduplicated(boolean isDeduplicated) { + this.isDeduplicated = isDeduplicated; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java index 83e664b..67d6c33 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java @@ -30,6 +30,10 @@ private int numPartitions; + public SparkEdgeProperty(){ + + } + public SparkEdgeProperty(long edgeType, int numPartitions) { this.edgeType = edgeType; this.numPartitions = numPartitions; @@ -91,7 +95,6 @@ public String getShuffleType() { sb.append(" "); } sb.append("PARTITION-LEVEL SORT"); - return sb.toString(); } if (isShuffleSort()) { @@ -99,7 +102,6 @@ public String getShuffleType() { sb.append(" "); } sb.append("SORT"); - return sb.toString(); } return sb.toString(); diff --git ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out index c0e1f7d..2165969 100644 --- ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out +++ ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out @@ -162,7 +162,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -237,7 +237,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -310,7 +310,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -384,7 +384,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -461,7 +461,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -550,7 +550,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -641,7 +641,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join0.q.out ql/src/test/results/clientpositive/spark/auto_join0.q.out index 76ff63d..1ca917e 100644 --- ql/src/test/results/clientpositive/spark/auto_join0.q.out +++ ql/src/test/results/clientpositive/spark/auto_join0.q.out @@ -28,8 +28,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join1.q.out ql/src/test/results/clientpositive/spark/auto_join1.q.out index 9dff27e..ea4364a 100644 --- ql/src/test/results/clientpositive/spark/auto_join1.q.out +++ ql/src/test/results/clientpositive/spark/auto_join1.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join10.q.out ql/src/test/results/clientpositive/spark/auto_join10.q.out index 05a5912..1ffb8e0 100644 --- ql/src/test/results/clientpositive/spark/auto_join10.q.out +++ ql/src/test/results/clientpositive/spark/auto_join10.q.out @@ -22,7 +22,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join11.q.out ql/src/test/results/clientpositive/spark/auto_join11.q.out index 6988a6e..8299359 100644 --- ql/src/test/results/clientpositive/spark/auto_join11.q.out +++ ql/src/test/results/clientpositive/spark/auto_join11.q.out @@ -22,7 +22,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join12.q.out ql/src/test/results/clientpositive/spark/auto_join12.q.out index 3558b50..7a9610f 100644 --- ql/src/test/results/clientpositive/spark/auto_join12.q.out +++ ql/src/test/results/clientpositive/spark/auto_join12.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join13.q.out ql/src/test/results/clientpositive/spark/auto_join13.q.out index 6195c01..a7f03e1 100644 --- ql/src/test/results/clientpositive/spark/auto_join13.q.out +++ ql/src/test/results/clientpositive/spark/auto_join13.q.out @@ -28,8 +28,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join14.q.out ql/src/test/results/clientpositive/spark/auto_join14.q.out index 9883485..f8c2f71 100644 --- ql/src/test/results/clientpositive/spark/auto_join14.q.out +++ ql/src/test/results/clientpositive/spark/auto_join14.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join15.q.out ql/src/test/results/clientpositive/spark/auto_join15.q.out index 5916070..5202b2d 100644 --- ql/src/test/results/clientpositive/spark/auto_join15.q.out +++ ql/src/test/results/clientpositive/spark/auto_join15.q.out @@ -22,8 +22,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join16.q.out ql/src/test/results/clientpositive/spark/auto_join16.q.out index 3913832..e5cecc4 100644 --- ql/src/test/results/clientpositive/spark/auto_join16.q.out +++ ql/src/test/results/clientpositive/spark/auto_join16.q.out @@ -22,7 +22,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join17.q.out ql/src/test/results/clientpositive/spark/auto_join17.q.out index 0a852c7..9d9915a 100644 --- ql/src/test/results/clientpositive/spark/auto_join17.q.out +++ ql/src/test/results/clientpositive/spark/auto_join17.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join18.q.out ql/src/test/results/clientpositive/spark/auto_join18.q.out index 6083b38..2d347a3 100644 --- ql/src/test/results/clientpositive/spark/auto_join18.q.out +++ ql/src/test/results/clientpositive/spark/auto_join18.q.out @@ -32,8 +32,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) Reducer 6 <- Map 5 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out index 01c8f0a..1a6cc33 100644 --- ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out +++ ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out @@ -34,8 +34,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) Reducer 6 <- Map 5 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/auto_join19.q.out ql/src/test/results/clientpositive/spark/auto_join19.q.out index fce4982..f716db1 100644 --- ql/src/test/results/clientpositive/spark/auto_join19.q.out +++ ql/src/test/results/clientpositive/spark/auto_join19.q.out @@ -26,7 +26,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join2.q.out ql/src/test/results/clientpositive/spark/auto_join2.q.out index 463efde..72bd25e 100644 --- ql/src/test/results/clientpositive/spark/auto_join2.q.out +++ ql/src/test/results/clientpositive/spark/auto_join2.q.out @@ -24,8 +24,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join20.q.out ql/src/test/results/clientpositive/spark/auto_join20.q.out index a8f2b9a..4dbf97a 100644 --- ql/src/test/results/clientpositive/spark/auto_join20.q.out +++ ql/src/test/results/clientpositive/spark/auto_join20.q.out @@ -22,8 +22,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -173,8 +173,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join21.q.out ql/src/test/results/clientpositive/spark/auto_join21.q.out index f9ac35d..1f2443b 100644 --- ql/src/test/results/clientpositive/spark/auto_join21.q.out +++ ql/src/test/results/clientpositive/spark/auto_join21.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join22.q.out ql/src/test/results/clientpositive/spark/auto_join22.q.out index 516322c..c6729c6 100644 --- ql/src/test/results/clientpositive/spark/auto_join22.q.out +++ ql/src/test/results/clientpositive/spark/auto_join22.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join23.q.out ql/src/test/results/clientpositive/spark/auto_join23.q.out index ce5a670..4c167a4 100644 --- ql/src/test/results/clientpositive/spark/auto_join23.q.out +++ ql/src/test/results/clientpositive/spark/auto_join23.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join24.q.out ql/src/test/results/clientpositive/spark/auto_join24.q.out index 7d75479..f80a863 100644 --- ql/src/test/results/clientpositive/spark/auto_join24.q.out +++ ql/src/test/results/clientpositive/spark/auto_join24.q.out @@ -32,7 +32,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join26.q.out ql/src/test/results/clientpositive/spark/auto_join26.q.out index fd94861..05ac213 100644 --- ql/src/test/results/clientpositive/spark/auto_join26.q.out +++ ql/src/test/results/clientpositive/spark/auto_join26.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join27.q.out ql/src/test/results/clientpositive/spark/auto_join27.q.out index 651f30f..ac8cc21 100644 --- ql/src/test/results/clientpositive/spark/auto_join27.q.out +++ ql/src/test/results/clientpositive/spark/auto_join27.q.out @@ -31,7 +31,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 1), Union 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Union 3 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Reducer 4 (GROUP, 1) Union 3 <- Map 6 (NONE, 0), Reducer 2 (NONE, 0) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/auto_join28.q.out ql/src/test/results/clientpositive/spark/auto_join28.q.out index b979661..f11e089 100644 --- ql/src/test/results/clientpositive/spark/auto_join28.q.out +++ ql/src/test/results/clientpositive/spark/auto_join28.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -110,8 +110,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -211,8 +211,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -312,8 +312,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join29.q.out ql/src/test/results/clientpositive/spark/auto_join29.q.out index 42043d3..0e5c4ac 100644 --- ql/src/test/results/clientpositive/spark/auto_join29.q.out +++ ql/src/test/results/clientpositive/spark/auto_join29.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -618,8 +618,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1227,8 +1227,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1848,8 +1848,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2466,8 +2466,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2574,8 +2574,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3183,8 +3183,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3292,8 +3292,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3426,8 +3426,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join3.q.out ql/src/test/results/clientpositive/spark/auto_join3.q.out index 3749908..1d7756a 100644 --- ql/src/test/results/clientpositive/spark/auto_join3.q.out +++ ql/src/test/results/clientpositive/spark/auto_join3.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join30.q.out ql/src/test/results/clientpositive/spark/auto_join30.q.out index 98b3974..6d44aec 100644 --- ql/src/test/results/clientpositive/spark/auto_join30.q.out +++ ql/src/test/results/clientpositive/spark/auto_join30.q.out @@ -22,10 +22,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -175,10 +175,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -322,10 +322,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -475,11 +475,11 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) - Reducer 8 <- Map 7 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) + Reducer 8 <- Map 7 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -671,11 +671,11 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) - Reducer 8 <- Map 7 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) + Reducer 8 <- Map 7 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -858,11 +858,11 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) - Reducer 8 <- Map 7 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) + Reducer 8 <- Map 7 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1045,11 +1045,11 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) - Reducer 8 <- Map 7 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) + Reducer 8 <- Map 7 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1232,11 +1232,11 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) - Reducer 8 <- Map 7 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) + Reducer 8 <- Map 7 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join31.q.out ql/src/test/results/clientpositive/spark/auto_join31.q.out index df502c8..57a7b8f 100644 --- ql/src/test/results/clientpositive/spark/auto_join31.q.out +++ ql/src/test/results/clientpositive/spark/auto_join31.q.out @@ -28,11 +28,11 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP SORT, 1) - Reducer 8 <- Map 7 (GROUP SORT, 1) + Reducer 6 <- Map 5 (SORT, 1) + Reducer 8 <- Map 7 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join32.q.out ql/src/test/results/clientpositive/spark/auto_join32.q.out index ca5c412..a5f0d1f 100644 --- ql/src/test/results/clientpositive/spark/auto_join32.q.out +++ ql/src/test/results/clientpositive/spark/auto_join32.q.out @@ -34,8 +34,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -168,7 +168,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -290,7 +290,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -436,7 +436,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join4.q.out ql/src/test/results/clientpositive/spark/auto_join4.q.out index 6faad30..20dff67 100644 --- ql/src/test/results/clientpositive/spark/auto_join4.q.out +++ ql/src/test/results/clientpositive/spark/auto_join4.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join5.q.out ql/src/test/results/clientpositive/spark/auto_join5.q.out index 8455f02..5158229 100644 --- ql/src/test/results/clientpositive/spark/auto_join5.q.out +++ ql/src/test/results/clientpositive/spark/auto_join5.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join6.q.out ql/src/test/results/clientpositive/spark/auto_join6.q.out index b5e0687..f046784 100644 --- ql/src/test/results/clientpositive/spark/auto_join6.q.out +++ ql/src/test/results/clientpositive/spark/auto_join6.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join7.q.out ql/src/test/results/clientpositive/spark/auto_join7.q.out index 4d42841..d11fea1 100644 --- ql/src/test/results/clientpositive/spark/auto_join7.q.out +++ ql/src/test/results/clientpositive/spark/auto_join7.q.out @@ -56,7 +56,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join8.q.out ql/src/test/results/clientpositive/spark/auto_join8.q.out index 4a8b765..183d24f 100644 --- ql/src/test/results/clientpositive/spark/auto_join8.q.out +++ ql/src/test/results/clientpositive/spark/auto_join8.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join9.q.out ql/src/test/results/clientpositive/spark/auto_join9.q.out index cf0d086..7694ca3 100644 --- ql/src/test/results/clientpositive/spark/auto_join9.q.out +++ ql/src/test/results/clientpositive/spark/auto_join9.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out index 96b329e..6f9e51a 100644 --- ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out +++ ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out @@ -175,10 +175,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 1), Reducer 3 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 9 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 9 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out index 00ab144..f5e8f88 100644 --- ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out +++ ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out @@ -12,7 +12,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -136,8 +136,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -290,8 +290,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out index 18afdbf..d961067 100644 --- ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out +++ ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out @@ -301,7 +301,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -956,7 +956,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out index ba13d81..8625f07 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out @@ -72,7 +72,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Union 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Union 2 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) Union 2 <- Map 1 (NONE, 0), Map 5 (NONE, 0) #### A masked pattern was here #### @@ -227,7 +227,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_11.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_11.q.out index 59f85e1..46c1e36 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_11.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_11.q.out @@ -145,7 +145,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -467,7 +467,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -788,7 +788,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1124,7 +1124,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out index 4212281..03ffb12 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out @@ -210,9 +210,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) - Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out index 58876c3..4562430 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out @@ -96,7 +96,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -204,7 +204,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -312,7 +312,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -416,7 +416,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -613,7 +613,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -717,7 +717,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -821,7 +821,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1018,7 +1018,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out index f5d3d55..063590f 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out @@ -406,7 +406,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1061,7 +1061,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1946,7 +1946,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucket4.q.out ql/src/test/results/clientpositive/spark/bucket4.q.out index a7e5921..7e86497 100644 --- ql/src/test/results/clientpositive/spark/bucket4.q.out +++ ql/src/test/results/clientpositive/spark/bucket4.q.out @@ -41,7 +41,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out index e14b5b0..ce121ac 100644 --- ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out +++ ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out @@ -111,7 +111,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out index 7a0f070..8c12ce7 100644 --- ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out +++ ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out @@ -111,7 +111,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out index 320c935..20b5adc 100644 --- ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out +++ ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out @@ -124,7 +124,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -205,7 +205,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -311,8 +311,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -447,8 +447,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -583,7 +583,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -673,8 +673,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -786,7 +786,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -892,7 +892,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -992,7 +992,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1088,7 +1088,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1159,8 +1159,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out index e07e8b6..e7ec4c8 100644 --- ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out +++ ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out @@ -120,8 +120,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -247,7 +247,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -320,7 +320,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -399,7 +399,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -478,7 +478,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -555,7 +555,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -650,7 +650,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out index aafac96..f632a0e 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out @@ -98,7 +98,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -234,7 +234,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -465,7 +465,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -854,7 +854,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out index 0464adb..4188ad8 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out @@ -199,7 +199,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out index 6ed7e8c..e4a98ba 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out @@ -209,7 +209,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -612,7 +612,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out index 8f52751..dbdf2b2 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out @@ -168,7 +168,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -464,7 +464,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out index 6df0df6..ebf1766 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out @@ -127,7 +127,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -467,7 +467,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -758,7 +758,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1051,7 +1051,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out index b80dd4b..a771a56 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out @@ -167,7 +167,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -565,7 +565,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -985,7 +985,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out index 63ff89a..520eb47 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out @@ -198,7 +198,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -603,7 +603,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out index 57cf2b5..c7ed4d5 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out @@ -184,7 +184,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -553,7 +553,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out index a030dd2..57b1fc7 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out @@ -234,7 +234,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -668,7 +668,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out index bd5a64c..2db3388 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out @@ -134,7 +134,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out index 10d308a..93c27cb 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out @@ -133,7 +133,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -438,7 +438,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out index b04f691..77d9fb7 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out @@ -141,7 +141,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -479,7 +479,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin_negative.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin_negative.q.out index ccbc79f..20e87d8 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin_negative.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin_negative.q.out @@ -142,7 +142,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin_negative2.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin_negative2.q.out index ab93bf6..1d6b68a 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin_negative2.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin_negative2.q.out @@ -144,7 +144,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin_negative3.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin_negative3.q.out index 47d8903..9b7cc22 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin_negative3.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin_negative3.q.out @@ -202,7 +202,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -438,7 +438,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -671,7 +671,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -909,7 +909,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1145,7 +1145,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1381,7 +1381,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1617,7 +1617,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1853,7 +1853,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2089,7 +2089,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/column_access_stats.q.out ql/src/test/results/clientpositive/spark/column_access_stats.q.out index 42f6629..8c281be 100644 --- ql/src/test/results/clientpositive/spark/column_access_stats.q.out +++ ql/src/test/results/clientpositive/spark/column_access_stats.q.out @@ -379,7 +379,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -502,7 +502,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -592,7 +592,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -704,8 +704,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/count.q.out ql/src/test/results/clientpositive/spark/count.q.out index c527c1d..89d5544 100644 --- ql/src/test/results/clientpositive/spark/count.q.out +++ ql/src/test/results/clientpositive/spark/count.q.out @@ -41,7 +41,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -115,7 +115,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -184,7 +184,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -252,7 +252,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/cross_join.q.out ql/src/test/results/clientpositive/spark/cross_join.q.out index 4a7f6a7..4e28710 100644 --- ql/src/test/results/clientpositive/spark/cross_join.q.out +++ ql/src/test/results/clientpositive/spark/cross_join.q.out @@ -12,7 +12,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -74,7 +74,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -136,7 +136,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ctas.q.out ql/src/test/results/clientpositive/spark/ctas.q.out index 158ec95..a042224 100644 --- ql/src/test/results/clientpositive/spark/ctas.q.out +++ ql/src/test/results/clientpositive/spark/ctas.q.out @@ -33,8 +33,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -181,8 +181,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -329,8 +329,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -542,8 +542,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -733,8 +733,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out index c98a14e..4622712 100644 --- ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out +++ ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out @@ -203,7 +203,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out index 52bdf6a..8bbd785 100644 --- ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out +++ ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out @@ -14,7 +14,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -65,7 +65,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/escape_orderby1.q.out ql/src/test/results/clientpositive/spark/escape_orderby1.q.out index 6e1c0cf..83a8f4f 100644 --- ql/src/test/results/clientpositive/spark/escape_orderby1.q.out +++ ql/src/test/results/clientpositive/spark/escape_orderby1.q.out @@ -14,7 +14,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -64,7 +64,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/escape_sortby1.q.out ql/src/test/results/clientpositive/spark/escape_sortby1.q.out index 58b663c..d872c9a 100644 --- ql/src/test/results/clientpositive/spark/escape_sortby1.q.out +++ ql/src/test/results/clientpositive/spark/escape_sortby1.q.out @@ -14,7 +14,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -64,7 +64,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby1.q.out ql/src/test/results/clientpositive/spark/groupby1.q.out index a337a39..46c45a3 100644 --- ql/src/test/results/clientpositive/spark/groupby1.q.out +++ ql/src/test/results/clientpositive/spark/groupby1.q.out @@ -26,7 +26,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby10.q.out ql/src/test/results/clientpositive/spark/groupby10.q.out index d2198de..561f1dd 100644 --- ql/src/test/results/clientpositive/spark/groupby10.q.out +++ ql/src/test/results/clientpositive/spark/groupby10.q.out @@ -56,7 +56,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### @@ -268,7 +268,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### @@ -482,7 +482,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby11.q.out ql/src/test/results/clientpositive/spark/groupby11.q.out index 730a4f3..41967e5 100644 --- ql/src/test/results/clientpositive/spark/groupby11.q.out +++ ql/src/test/results/clientpositive/spark/groupby11.q.out @@ -44,7 +44,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby2.q.out ql/src/test/results/clientpositive/spark/groupby2.q.out index 1408571..1ec371e 100644 --- ql/src/test/results/clientpositive/spark/groupby2.q.out +++ ql/src/test/results/clientpositive/spark/groupby2.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby3.q.out ql/src/test/results/clientpositive/spark/groupby3.q.out index 31c816f..b8219c4 100644 --- ql/src/test/results/clientpositive/spark/groupby3.q.out +++ ql/src/test/results/clientpositive/spark/groupby3.q.out @@ -42,7 +42,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby3_map.q.out ql/src/test/results/clientpositive/spark/groupby3_map.q.out index f8b5b88..7728c3b 100644 --- ql/src/test/results/clientpositive/spark/groupby3_map.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_map.q.out @@ -42,7 +42,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out index 03a80be..1ca3252 100644 --- ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out index 1778186..250cd24 100644 --- ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out @@ -42,7 +42,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -159,4 +159,4 @@ POSTHOOK: query: SELECT c1, c2, c3, c4, c5, c6, c7, ROUND(c8, 5), ROUND(c9, 5) F POSTHOOK: type: QUERY POSTHOOK: Input: default@dest1 #### A masked pattern was here #### -130091.0 260.182 256.10355987055016 98.0 0.0 142.92680950752379 143.06995106518903 20428.07288 20469.0109 +130091.0 260.182 256.10355987055016 98.0 0.0 142.9268095075238 143.06995106518906 20428.07288 20469.0109 diff --git ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out index 2343bdd..e06d56b 100644 --- ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out @@ -42,7 +42,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out index bc51624..268a1d7 100644 --- ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby4.q.out ql/src/test/results/clientpositive/spark/groupby4.q.out index 38e4df7..04fba3d 100644 --- ql/src/test/results/clientpositive/spark/groupby4.q.out +++ ql/src/test/results/clientpositive/spark/groupby4.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out index 8dbbe19..7ba2e1d 100644 --- ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (SORT, 31) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out index ed610cf..2e43287 100644 --- ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out @@ -40,9 +40,9 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 31) - Reducer 4 <- Map 1 (GROUP SORT, 31) + Reducer 4 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) Reducer 5 <- Reducer 4 (GROUP, 31) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out index 9d94b0e..396c6a4 100644 --- ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby8.q.out ql/src/test/results/clientpositive/spark/groupby8.q.out index 10cd709..d7c5b6d 100644 --- ql/src/test/results/clientpositive/spark/groupby8.q.out +++ ql/src/test/results/clientpositive/spark/groupby8.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### @@ -828,7 +828,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby8_map.q.out ql/src/test/results/clientpositive/spark/groupby8_map.q.out index 4d4152c..9d491be 100644 --- ql/src/test/results/clientpositive/spark/groupby8_map.q.out +++ ql/src/test/results/clientpositive/spark/groupby8_map.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 31) Reducer 4 <- Reducer 2 (GROUP, 31) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out index 4d4152c..9d491be 100644 --- ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out +++ ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 31) Reducer 4 <- Reducer 2 (GROUP, 31) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out index 4d4152c..9d491be 100644 --- ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out +++ ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 31) Reducer 4 <- Reducer 2 (GROUP, 31) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby9.q.out ql/src/test/results/clientpositive/spark/groupby9.q.out index 8cb1e8a..de795a0 100644 --- ql/src/test/results/clientpositive/spark/groupby9.q.out +++ ql/src/test/results/clientpositive/spark/groupby9.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### @@ -829,7 +829,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### @@ -1618,7 +1618,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### @@ -3193,7 +3193,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/groupby_cube1.q.out ql/src/test/results/clientpositive/spark/groupby_cube1.q.out index 01a6b48..0d0f922 100644 --- ql/src/test/results/clientpositive/spark/groupby_cube1.q.out +++ ql/src/test/results/clientpositive/spark/groupby_cube1.q.out @@ -122,7 +122,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -199,8 +199,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -304,7 +304,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -405,10 +405,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Map 1 (GROUP SORT, 1) - Reducer 5 <- Reducer 4 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out new file mode 100644 index 0000000..a950e7f --- /dev/null +++ ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out @@ -0,0 +1,342 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest1 +PREHOOK: query: EXPLAIN EXTENDED +FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN EXTENDED +FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + srcpart + src + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + dest1 + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + key + 1 + 1 + TOK_SELEXPR + TOK_FUNCTIONDI + count + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + value + 5 + TOK_SELEXPR + TOK_FUNCTION + concat + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + key + 1 + 1 + TOK_FUNCTION + sum + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + value + 5 + TOK_WHERE + = + . + TOK_TABLE_OR_COL + src + ds + '2008-04-08' + TOK_GROUPBY + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + key + 1 + 1 + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: key, value + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(DISTINCT substr(value, 5)), sum(substr(value, 5)) + keys: substr(key, 1, 1) (type: string), substr(value, 5) (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col3 (type: double) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 500 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 5312 + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart +#### A masked pattern was here #### + Partition + base file name: hr=12 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 12 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 500 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 5312 + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [src] + /srcpart/ds=2008-04-08/hr=12 [src] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0), sum(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,c1,c2 + columns.comments + columns.types string:int:string +#### A masked pattern was here #### + name default.dest1 + serialization.ddl struct dest1 { string key, i32 c1, string c2} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest1 + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,c1,c2 + columns.comments + columns.types string:int:string +#### A masked pattern was here #### + name default.dest1 + serialization.ddl struct dest1 { string key, i32 c1, string c2} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest1 + + Stage: Stage-3 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@dest1 +POSTHOOK: query: FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@dest1 +POSTHOOK: Lineage: dest1.c1 EXPRESSION [(srcpart)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.c2 EXPRESSION [(srcpart)src.FieldSchema(name:key, type:string, comment:default), (srcpart)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.key EXPRESSION [(srcpart)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: SELECT dest1.* FROM dest1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT dest1.* FROM dest1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest1 +#### A masked pattern was here #### +0 1 00.0 +1 71 132828.0 +2 69 251142.0 +3 62 364008.0 +4 74 4105526.0 +5 6 5794.0 +6 5 6796.0 +7 6 71470.0 +8 8 81524.0 +9 7 92094.0 diff --git ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out new file mode 100644 index 0000000..3592169 --- /dev/null +++ ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out @@ -0,0 +1,361 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, C3 INT, c4 INT) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, C3 INT, c4 INT) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest1 +PREHOOK: query: EXPLAIN EXTENDED +FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN EXTENDED +FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + srcpart + src + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + dest1 + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + key + 1 + 1 + TOK_SELEXPR + TOK_FUNCTIONDI + count + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + value + 5 + TOK_SELEXPR + TOK_FUNCTION + concat + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + key + 1 + 1 + TOK_FUNCTION + sum + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + value + 5 + TOK_SELEXPR + TOK_FUNCTIONDI + sum + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + value + 5 + TOK_SELEXPR + TOK_FUNCTIONDI + count + . + TOK_TABLE_OR_COL + src + value + TOK_WHERE + = + . + TOK_TABLE_OR_COL + src + ds + '2008-04-08' + TOK_GROUPBY + TOK_FUNCTION + substr + . + TOK_TABLE_OR_COL + src + key + 1 + 1 + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: key, value + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(DISTINCT substr(value, 5)), sum(substr(value, 5)), sum(DISTINCT substr(value, 5)), count(DISTINCT value) + keys: substr(key, 1, 1) (type: string), substr(value, 5) (type: string), value (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col4 (type: double) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 500 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 5312 + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart +#### A masked pattern was here #### + Partition + base file name: hr=12 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 12 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 500 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 5312 + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [src] + /srcpart/ds=2008-04-08/hr=12 [src] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0), sum(VALUE._col1), sum(DISTINCT KEY._col1:1._col0), count(DISTINCT KEY._col1:2._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,c1,c2,c3,c4 + columns.comments + columns.types string:int:string:int:int +#### A masked pattern was here #### + name default.dest1 + serialization.ddl struct dest1 { string key, i32 c1, string c2, i32 c3, i32 c4} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest1 + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,c1,c2,c3,c4 + columns.comments + columns.types string:int:string:int:int +#### A masked pattern was here #### + name default.dest1 + serialization.ddl struct dest1 { string key, i32 c1, string c2, i32 c3, i32 c4} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest1 + + Stage: Stage-3 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@dest1 +POSTHOOK: query: FROM srcpart src +INSERT OVERWRITE TABLE dest1 +SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@dest1 +POSTHOOK: Lineage: dest1.c1 EXPRESSION [(srcpart)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.c2 EXPRESSION [(srcpart)src.FieldSchema(name:key, type:string, comment:default), (srcpart)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.c3 EXPRESSION [(srcpart)src.null, ] +POSTHOOK: Lineage: dest1.c4 EXPRESSION [(srcpart)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.key EXPRESSION [(srcpart)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: SELECT dest1.* FROM dest1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT dest1.* FROM dest1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest1 +#### A masked pattern was here #### +0 1 00.0 0 1 +1 71 132828.0 10044 71 +2 69 251142.0 15780 69 +3 62 364008.0 20119 62 +4 74 4105526.0 30965 74 +5 6 5794.0 278 6 +6 5 6796.0 331 5 +7 6 71470.0 447 6 +8 8 81524.0 595 8 +9 7 92094.0 577 7 diff --git ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out index 5f7b87a..c1fd88d 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### @@ -224,8 +224,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out index df10b08..2dab62f 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out @@ -68,7 +68,7 @@ STAGE PLANS: Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -304,8 +304,8 @@ STAGE PLANS: Stage: Stage-5 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 1 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out index c944e19..5b7eb9b 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out @@ -36,7 +36,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out index a137c46..2cd4d37 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out @@ -50,7 +50,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -230,7 +230,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -410,7 +410,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -590,7 +590,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby_position.q.out ql/src/test/results/clientpositive/spark/groupby_position.q.out index c6fd342..697184b 100644 --- ql/src/test/results/clientpositive/spark/groupby_position.q.out +++ ql/src/test/results/clientpositive/spark/groupby_position.q.out @@ -40,8 +40,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -238,8 +238,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -437,7 +437,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -566,9 +566,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (SORT, 1) Reducer 6 <- Map 5 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby_ppr.q.out ql/src/test/results/clientpositive/spark/groupby_ppr.q.out index a0069a2..565c716 100644 --- ql/src/test/results/clientpositive/spark/groupby_ppr.q.out +++ ql/src/test/results/clientpositive/spark/groupby_ppr.q.out @@ -101,7 +101,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out index 4af8f38..eb87c7e 100644 --- ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out +++ ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out @@ -116,7 +116,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -193,8 +193,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -292,7 +292,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -393,10 +393,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Map 1 (GROUP SORT, 1) - Reducer 5 <- Reducer 4 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out index 8d237c5..2cbe0d1 100644 --- ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out +++ ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out @@ -2773,7 +2773,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3156,7 +3156,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out index 4946815..c599f4c 100644 --- ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out +++ ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out @@ -302,7 +302,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1223,7 +1223,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1472,7 +1472,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1757,7 +1757,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -2432,7 +2432,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1) Reducer 5 <- Reducer 4 (GROUP, 1) Union 2 <- Map 1 (NONE, 0), Reducer 5 (NONE, 0) #### A masked pattern was here #### @@ -2863,7 +2863,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3246,8 +3246,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -3565,7 +3565,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -4788,7 +4788,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 31) #### A masked pattern was here #### Vertices: @@ -4970,7 +4970,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 31) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) Reducer 3 <- Reducer 2 (GROUP, 31) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/index_auto_self_join.q.out ql/src/test/results/clientpositive/spark/index_auto_self_join.q.out index 537847b..5cdd660 100644 --- ql/src/test/results/clientpositive/spark/index_auto_self_join.q.out +++ ql/src/test/results/clientpositive/spark/index_auto_self_join.q.out @@ -16,7 +16,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -126,7 +126,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/innerjoin.q.out ql/src/test/results/clientpositive/spark/innerjoin.q.out index acfe946..798ca32 100644 --- ql/src/test/results/clientpositive/spark/innerjoin.q.out +++ ql/src/test/results/clientpositive/spark/innerjoin.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1193,7 +1193,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/input14.q.out ql/src/test/results/clientpositive/spark/input14.q.out index 676caec..17b1b39 100644 --- ql/src/test/results/clientpositive/spark/input14.q.out +++ ql/src/test/results/clientpositive/spark/input14.q.out @@ -34,7 +34,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/input17.q.out ql/src/test/results/clientpositive/spark/input17.q.out index 8922f15..aa4eec4 100644 --- ql/src/test/results/clientpositive/spark/input17.q.out +++ ql/src/test/results/clientpositive/spark/input17.q.out @@ -34,7 +34,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/input18.q.out ql/src/test/results/clientpositive/spark/input18.q.out index 46df37a..81f88de 100644 --- ql/src/test/results/clientpositive/spark/input18.q.out +++ ql/src/test/results/clientpositive/spark/input18.q.out @@ -34,7 +34,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/insert_into3.q.out ql/src/test/results/clientpositive/spark/insert_into3.q.out index 04c9858..1313d26 100644 --- ql/src/test/results/clientpositive/spark/insert_into3.q.out +++ ql/src/test/results/clientpositive/spark/insert_into3.q.out @@ -40,8 +40,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join0.q.out ql/src/test/results/clientpositive/spark/join0.q.out index 55b725e..b67b4f1 100644 --- ql/src/test/results/clientpositive/spark/join0.q.out +++ ql/src/test/results/clientpositive/spark/join0.q.out @@ -22,8 +22,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join1.q.out ql/src/test/results/clientpositive/spark/join1.q.out index ec726c3..fee68c0 100644 --- ql/src/test/results/clientpositive/spark/join1.q.out +++ ql/src/test/results/clientpositive/spark/join1.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join10.q.out ql/src/test/results/clientpositive/spark/join10.q.out index f19f349..9dd4e59 100644 --- ql/src/test/results/clientpositive/spark/join10.q.out +++ ql/src/test/results/clientpositive/spark/join10.q.out @@ -20,7 +20,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join11.q.out ql/src/test/results/clientpositive/spark/join11.q.out index f982de4..bce7b2b 100644 --- ql/src/test/results/clientpositive/spark/join11.q.out +++ ql/src/test/results/clientpositive/spark/join11.q.out @@ -22,7 +22,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join12.q.out ql/src/test/results/clientpositive/spark/join12.q.out index a6988b7..38a42ff 100644 --- ql/src/test/results/clientpositive/spark/join12.q.out +++ ql/src/test/results/clientpositive/spark/join12.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join13.q.out ql/src/test/results/clientpositive/spark/join13.q.out index 52b7a54..1f3374d 100644 --- ql/src/test/results/clientpositive/spark/join13.q.out +++ ql/src/test/results/clientpositive/spark/join13.q.out @@ -28,8 +28,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join14.q.out ql/src/test/results/clientpositive/spark/join14.q.out index 6ad27d3..4046e68 100644 --- ql/src/test/results/clientpositive/spark/join14.q.out +++ ql/src/test/results/clientpositive/spark/join14.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join15.q.out ql/src/test/results/clientpositive/spark/join15.q.out index 1651db1..9e9cd09 100644 --- ql/src/test/results/clientpositive/spark/join15.q.out +++ ql/src/test/results/clientpositive/spark/join15.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join16.q.out ql/src/test/results/clientpositive/spark/join16.q.out index 8b6a7e6..34a4cb1 100644 --- ql/src/test/results/clientpositive/spark/join16.q.out +++ ql/src/test/results/clientpositive/spark/join16.q.out @@ -10,7 +10,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join17.q.out ql/src/test/results/clientpositive/spark/join17.q.out index c151493..a4d016f 100644 --- ql/src/test/results/clientpositive/spark/join17.q.out +++ ql/src/test/results/clientpositive/spark/join17.q.out @@ -62,7 +62,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join18.q.out ql/src/test/results/clientpositive/spark/join18.q.out index 7b64fb6..f16fc04 100644 --- ql/src/test/results/clientpositive/spark/join18.q.out +++ ql/src/test/results/clientpositive/spark/join18.q.out @@ -32,8 +32,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out index 57c4516..2960522 100644 --- ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out +++ ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out @@ -38,8 +38,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join19.q.out ql/src/test/results/clientpositive/spark/join19.q.out index eb1cd6b..5220349 100644 --- ql/src/test/results/clientpositive/spark/join19.q.out +++ ql/src/test/results/clientpositive/spark/join19.q.out @@ -126,9 +126,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 7 (GROUP PARTITION-LEVEL SORT, 1), Map 9 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 7 (PARTITION-LEVEL SORT, 1), Map 9 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join2.q.out ql/src/test/results/clientpositive/spark/join2.q.out index 22f23c7..bff76ef 100644 --- ql/src/test/results/clientpositive/spark/join2.q.out +++ ql/src/test/results/clientpositive/spark/join2.q.out @@ -24,8 +24,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join20.q.out ql/src/test/results/clientpositive/spark/join20.q.out index f06ffac..ae40f1c 100644 --- ql/src/test/results/clientpositive/spark/join20.q.out +++ ql/src/test/results/clientpositive/spark/join20.q.out @@ -14,8 +14,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -675,8 +675,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join21.q.out ql/src/test/results/clientpositive/spark/join21.q.out index e81ec5a..7ca62a1 100644 --- ql/src/test/results/clientpositive/spark/join21.q.out +++ ql/src/test/results/clientpositive/spark/join21.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join22.q.out ql/src/test/results/clientpositive/spark/join22.q.out index 6361943..d108173 100644 --- ql/src/test/results/clientpositive/spark/join22.q.out +++ ql/src/test/results/clientpositive/spark/join22.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join23.q.out ql/src/test/results/clientpositive/spark/join23.q.out index 3982ea7..53d44d9 100644 --- ql/src/test/results/clientpositive/spark/join23.q.out +++ ql/src/test/results/clientpositive/spark/join23.q.out @@ -12,8 +12,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join25.q.out ql/src/test/results/clientpositive/spark/join25.q.out index 60a0675..9a5db27 100644 --- ql/src/test/results/clientpositive/spark/join25.q.out +++ ql/src/test/results/clientpositive/spark/join25.q.out @@ -30,7 +30,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join26.q.out ql/src/test/results/clientpositive/spark/join26.q.out index 249d896..e4ccf02 100644 --- ql/src/test/results/clientpositive/spark/join26.q.out +++ ql/src/test/results/clientpositive/spark/join26.q.out @@ -111,7 +111,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join27.q.out ql/src/test/results/clientpositive/spark/join27.q.out index 98bff59..81cc089 100644 --- ql/src/test/results/clientpositive/spark/join27.q.out +++ ql/src/test/results/clientpositive/spark/join27.q.out @@ -30,7 +30,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join28.q.out ql/src/test/results/clientpositive/spark/join28.q.out index 0a7c1de..0349069 100644 --- ql/src/test/results/clientpositive/spark/join28.q.out +++ ql/src/test/results/clientpositive/spark/join28.q.out @@ -40,8 +40,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join29.q.out ql/src/test/results/clientpositive/spark/join29.q.out index 6da549d..374f89c 100644 --- ql/src/test/results/clientpositive/spark/join29.q.out +++ ql/src/test/results/clientpositive/spark/join29.q.out @@ -37,7 +37,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join3.q.out ql/src/test/results/clientpositive/spark/join3.q.out index 662989c..c9ec3dc 100644 --- ql/src/test/results/clientpositive/spark/join3.q.out +++ ql/src/test/results/clientpositive/spark/join3.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join30.q.out ql/src/test/results/clientpositive/spark/join30.q.out index 8d7071e..94645bb 100644 --- ql/src/test/results/clientpositive/spark/join30.q.out +++ ql/src/test/results/clientpositive/spark/join30.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join31.q.out ql/src/test/results/clientpositive/spark/join31.q.out index 8492326..00c2cba 100644 --- ql/src/test/results/clientpositive/spark/join31.q.out +++ ql/src/test/results/clientpositive/spark/join31.q.out @@ -39,7 +39,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) Reducer 6 <- Map 5 (GROUP, 1) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/join32.q.out ql/src/test/results/clientpositive/spark/join32.q.out index d858849..8c538fc 100644 --- ql/src/test/results/clientpositive/spark/join32.q.out +++ ql/src/test/results/clientpositive/spark/join32.q.out @@ -109,8 +109,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join32_lessSize.q.out ql/src/test/results/clientpositive/spark/join32_lessSize.q.out index 2416eec..00d3c63 100644 --- ql/src/test/results/clientpositive/spark/join32_lessSize.q.out +++ ql/src/test/results/clientpositive/spark/join32_lessSize.q.out @@ -117,8 +117,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -626,8 +626,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1226,8 +1226,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1755,8 +1755,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2210,8 +2210,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2451,8 +2451,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join33.q.out ql/src/test/results/clientpositive/spark/join33.q.out index d858849..8c538fc 100644 --- ql/src/test/results/clientpositive/spark/join33.q.out +++ ql/src/test/results/clientpositive/spark/join33.q.out @@ -109,8 +109,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join34.q.out ql/src/test/results/clientpositive/spark/join34.q.out index 33066fa..ad52ccd 100644 --- ql/src/test/results/clientpositive/spark/join34.q.out +++ ql/src/test/results/clientpositive/spark/join34.q.out @@ -148,7 +148,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Union 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Union 2 (PARTITION-LEVEL SORT, 1) Union 2 <- Map 1 (NONE, 0), Map 4 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join35.q.out ql/src/test/results/clientpositive/spark/join35.q.out index 0fdb007..53ab4c7 100644 --- ql/src/test/results/clientpositive/spark/join35.q.out +++ ql/src/test/results/clientpositive/spark/join35.q.out @@ -157,7 +157,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 1), Union 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Union 3 (PARTITION-LEVEL SORT, 1) Reducer 6 <- Map 5 (GROUP, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/join36.q.out ql/src/test/results/clientpositive/spark/join36.q.out index 4828bd7..1608626 100644 --- ql/src/test/results/clientpositive/spark/join36.q.out +++ ql/src/test/results/clientpositive/spark/join36.q.out @@ -70,7 +70,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join37.q.out ql/src/test/results/clientpositive/spark/join37.q.out index 6d402ac..617be10 100644 --- ql/src/test/results/clientpositive/spark/join37.q.out +++ ql/src/test/results/clientpositive/spark/join37.q.out @@ -30,7 +30,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join38.q.out ql/src/test/results/clientpositive/spark/join38.q.out index bec5888..b792dad 100644 --- ql/src/test/results/clientpositive/spark/join38.q.out +++ ql/src/test/results/clientpositive/spark/join38.q.out @@ -56,7 +56,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join39.q.out ql/src/test/results/clientpositive/spark/join39.q.out index 80bcc47..a2bda84 100644 --- ql/src/test/results/clientpositive/spark/join39.q.out +++ ql/src/test/results/clientpositive/spark/join39.q.out @@ -30,7 +30,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join4.q.out ql/src/test/results/clientpositive/spark/join4.q.out index e56ebcf..4d019c7 100644 --- ql/src/test/results/clientpositive/spark/join4.q.out +++ ql/src/test/results/clientpositive/spark/join4.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join40.q.out ql/src/test/results/clientpositive/spark/join40.q.out index 8e04c49..0a96c44 100644 --- ql/src/test/results/clientpositive/spark/join40.q.out +++ ql/src/test/results/clientpositive/spark/join40.q.out @@ -16,7 +16,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -666,7 +666,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1778,8 +1778,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2439,8 +2439,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3100,7 +3100,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3750,7 +3750,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join41.q.out ql/src/test/results/clientpositive/spark/join41.q.out index cbc6e7a..630e406 100644 --- ql/src/test/results/clientpositive/spark/join41.q.out +++ ql/src/test/results/clientpositive/spark/join41.q.out @@ -22,7 +22,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -105,7 +105,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join5.q.out ql/src/test/results/clientpositive/spark/join5.q.out index eb60466..9aaf9f6 100644 --- ql/src/test/results/clientpositive/spark/join5.q.out +++ ql/src/test/results/clientpositive/spark/join5.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join6.q.out ql/src/test/results/clientpositive/spark/join6.q.out index 64c2221..c8b8a71 100644 --- ql/src/test/results/clientpositive/spark/join6.q.out +++ ql/src/test/results/clientpositive/spark/join6.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join7.q.out ql/src/test/results/clientpositive/spark/join7.q.out index cde86b2..9ed4520 100644 --- ql/src/test/results/clientpositive/spark/join7.q.out +++ ql/src/test/results/clientpositive/spark/join7.q.out @@ -56,7 +56,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join8.q.out ql/src/test/results/clientpositive/spark/join8.q.out index e413e77..79b0607 100644 --- ql/src/test/results/clientpositive/spark/join8.q.out +++ ql/src/test/results/clientpositive/spark/join8.q.out @@ -46,7 +46,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join9.q.out ql/src/test/results/clientpositive/spark/join9.q.out index 2834966..f1dd62e 100644 --- ql/src/test/results/clientpositive/spark/join9.q.out +++ ql/src/test/results/clientpositive/spark/join9.q.out @@ -78,7 +78,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out index 97bd917..95f9b1d 100644 --- ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out +++ ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out @@ -54,7 +54,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -119,7 +119,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -210,7 +210,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -305,8 +305,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -411,9 +411,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -550,9 +550,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out index 78b37ca..383b57e 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out @@ -54,7 +54,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -143,7 +143,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -232,8 +232,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -332,8 +332,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out index 0411dcd..93b929f 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out @@ -54,7 +54,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -161,9 +161,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out index 5b9799b..1af4ce8 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out @@ -56,7 +56,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -150,7 +150,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -244,8 +244,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -349,8 +349,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out index 00ef044..22786e8 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out @@ -56,7 +56,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -168,9 +168,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual1.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual1.q.out index 02afd2a..3306996 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual1.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual1.q.out @@ -110,7 +110,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -199,7 +199,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -288,8 +288,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -388,8 +388,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual2.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual2.q.out index 8f0390a..2fb05fe 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual2.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual2.q.out @@ -110,7 +110,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -217,9 +217,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual3.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual3.q.out index 85de5db..a3273c3 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual3.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual3.q.out @@ -112,7 +112,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -206,7 +206,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -300,8 +300,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -405,8 +405,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual4.q.out ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual4.q.out index 6e59ff2..9e68789 100644 --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual4.q.out +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual4.q.out @@ -112,7 +112,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -224,9 +224,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out index f2cfc5c..b729063 100644 --- ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out +++ ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out @@ -100,7 +100,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -459,7 +459,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -832,7 +832,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1219,7 +1219,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1664,7 +1664,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_hive_626.q.out ql/src/test/results/clientpositive/spark/join_hive_626.q.out index e2a12a6..f58f1ce 100644 --- ql/src/test/results/clientpositive/spark/join_hive_626.q.out +++ ql/src/test/results/clientpositive/spark/join_hive_626.q.out @@ -72,8 +72,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_map_ppr.q.out ql/src/test/results/clientpositive/spark/join_map_ppr.q.out index eabceb3..9269c08 100644 --- ql/src/test/results/clientpositive/spark/join_map_ppr.q.out +++ ql/src/test/results/clientpositive/spark/join_map_ppr.q.out @@ -113,7 +113,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -670,7 +670,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_merge_multi_expressions.q.out ql/src/test/results/clientpositive/spark/join_merge_multi_expressions.q.out index 8e924be..87afca1 100644 --- ql/src/test/results/clientpositive/spark/join_merge_multi_expressions.q.out +++ ql/src/test/results/clientpositive/spark/join_merge_multi_expressions.q.out @@ -12,7 +12,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/join_merging.q.out ql/src/test/results/clientpositive/spark/join_merging.q.out index e443ef0..e1ec29f 100644 --- ql/src/test/results/clientpositive/spark/join_merging.q.out +++ ql/src/test/results/clientpositive/spark/join_merging.q.out @@ -44,7 +44,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -130,7 +130,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_nullsafe.q.out ql/src/test/results/clientpositive/spark/join_nullsafe.q.out index 57d24b3..2ce052f 100644 --- ql/src/test/results/clientpositive/spark/join_nullsafe.q.out +++ ql/src/test/results/clientpositive/spark/join_nullsafe.q.out @@ -32,7 +32,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -117,7 +117,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -214,7 +214,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -330,7 +330,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -425,7 +425,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1564,7 +1564,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_rc.q.out ql/src/test/results/clientpositive/spark/join_rc.q.out index 12cc876..95b71fa 100644 --- ql/src/test/results/clientpositive/spark/join_rc.q.out +++ ql/src/test/results/clientpositive/spark/join_rc.q.out @@ -50,7 +50,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_reorder.q.out ql/src/test/results/clientpositive/spark/join_reorder.q.out index db01652..5ff56fb 100644 --- ql/src/test/results/clientpositive/spark/join_reorder.q.out +++ ql/src/test/results/clientpositive/spark/join_reorder.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -133,7 +133,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -242,8 +242,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -339,8 +339,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -482,7 +482,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -566,7 +566,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_reorder2.q.out ql/src/test/results/clientpositive/spark/join_reorder2.q.out index 1698575..ce57266 100644 --- ql/src/test/results/clientpositive/spark/join_reorder2.q.out +++ ql/src/test/results/clientpositive/spark/join_reorder2.q.out @@ -82,7 +82,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -214,9 +214,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_reorder3.q.out ql/src/test/results/clientpositive/spark/join_reorder3.q.out index ebcbd0d..16ad478 100644 --- ql/src/test/results/clientpositive/spark/join_reorder3.q.out +++ ql/src/test/results/clientpositive/spark/join_reorder3.q.out @@ -82,7 +82,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -214,9 +214,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_reorder4.q.out ql/src/test/results/clientpositive/spark/join_reorder4.q.out index 4f9cf2f..d0eabe5 100644 --- ql/src/test/results/clientpositive/spark/join_reorder4.q.out +++ ql/src/test/results/clientpositive/spark/join_reorder4.q.out @@ -58,7 +58,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -158,7 +158,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -258,7 +258,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_star.q.out ql/src/test/results/clientpositive/spark/join_star.q.out index d666adf..430b699 100644 --- ql/src/test/results/clientpositive/spark/join_star.q.out +++ ql/src/test/results/clientpositive/spark/join_star.q.out @@ -138,7 +138,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -229,8 +229,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -353,8 +353,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -477,8 +477,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -608,12 +608,12 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 12 (GROUP PARTITION-LEVEL SORT, 1), Map 14 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 13 (GROUP PARTITION-LEVEL SORT, 1), Reducer 3 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 11 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 9 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 9 <- Map 10 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 12 (PARTITION-LEVEL SORT, 1), Map 14 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 13 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 11 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 9 <- Map 10 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_thrift.q.out ql/src/test/results/clientpositive/spark/join_thrift.q.out index 38ffdd8..c0cb097 100644 --- ql/src/test/results/clientpositive/spark/join_thrift.q.out +++ ql/src/test/results/clientpositive/spark/join_thrift.q.out @@ -34,7 +34,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_vc.q.out ql/src/test/results/clientpositive/spark/join_vc.q.out index 6a6fad2..e4d4724 100644 --- ql/src/test/results/clientpositive/spark/join_vc.q.out +++ ql/src/test/results/clientpositive/spark/join_vc.q.out @@ -14,9 +14,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -143,7 +143,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/join_view.q.out ql/src/test/results/clientpositive/spark/join_view.q.out index ef7a31d..88ed110 100644 --- ql/src/test/results/clientpositive/spark/join_view.q.out +++ ql/src/test/results/clientpositive/spark/join_view.q.out @@ -50,7 +50,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/limit_pushdown.q.out ql/src/test/results/clientpositive/spark/limit_pushdown.q.out index 520c2aa..9ab8982 100644 --- ql/src/test/results/clientpositive/spark/limit_pushdown.q.out +++ ql/src/test/results/clientpositive/spark/limit_pushdown.q.out @@ -20,7 +20,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -103,7 +103,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -284,7 +284,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -475,7 +475,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -572,7 +572,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -696,7 +696,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -813,8 +813,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) Reducer 6 <- Reducer 5 (GROUP, 1) #### A masked pattern was here #### @@ -1064,7 +1064,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out index fd2a7be..88a0184 100644 --- ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out +++ ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out @@ -108,7 +108,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -489,7 +489,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -973,7 +973,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1452,7 +1452,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapjoin1.q.out ql/src/test/results/clientpositive/spark/mapjoin1.q.out index 4ccc8a0..2e9fdc9 100644 --- ql/src/test/results/clientpositive/spark/mapjoin1.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin1.q.out @@ -37,7 +37,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -130,7 +130,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -225,7 +225,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -322,7 +322,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -410,7 +410,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -506,7 +506,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapjoin_decimal.q.out ql/src/test/results/clientpositive/spark/mapjoin_decimal.q.out index d150e97..e5da4bf 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_decimal.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_decimal.q.out @@ -86,7 +86,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out index 9f66974..8813ccd 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out @@ -18,8 +18,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -159,7 +159,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -287,8 +287,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -423,7 +423,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/mapjoin_filter_on_outerjoin.q.out ql/src/test/results/clientpositive/spark/mapjoin_filter_on_outerjoin.q.out index e89ad85..cee0475 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_filter_on_outerjoin.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_filter_on_outerjoin.q.out @@ -60,8 +60,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -209,8 +209,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out index e71b0ca..03dab8f 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out @@ -59,8 +59,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -480,8 +480,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -581,8 +581,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out index e667137..50ba5f1 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out @@ -36,7 +36,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out index 3631911..46e6be4 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out @@ -26,8 +26,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -268,8 +268,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapjoin_subquery2.q.out ql/src/test/results/clientpositive/spark/mapjoin_subquery2.q.out index 4138a8a..69cbd58 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_subquery2.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_subquery2.q.out @@ -90,8 +90,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out index 4595431..7abce67 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out @@ -251,8 +251,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1077,8 +1077,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mapreduce1.q.out ql/src/test/results/clientpositive/spark/mapreduce1.q.out index d328632..60e7e41 100644 --- ql/src/test/results/clientpositive/spark/mapreduce1.q.out +++ ql/src/test/results/clientpositive/spark/mapreduce1.q.out @@ -32,7 +32,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mergejoins.q.out ql/src/test/results/clientpositive/spark/mergejoins.q.out index de88e0f..8ccf70a 100644 --- ql/src/test/results/clientpositive/spark/mergejoins.q.out +++ ql/src/test/results/clientpositive/spark/mergejoins.q.out @@ -50,8 +50,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -186,7 +186,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/mergejoins_mixed.q.out ql/src/test/results/clientpositive/spark/mergejoins_mixed.q.out index 1dda951..c667fdc 100644 --- ql/src/test/results/clientpositive/spark/mergejoins_mixed.q.out +++ ql/src/test/results/clientpositive/spark/mergejoins_mixed.q.out @@ -26,7 +26,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -119,7 +119,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -212,7 +212,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -305,7 +305,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -400,8 +400,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -508,8 +508,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -616,8 +616,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -726,9 +726,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -855,9 +855,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -984,9 +984,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1113,9 +1113,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1244,8 +1244,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/multi_insert.q.out ql/src/test/results/clientpositive/spark/multi_insert.q.out index 4b7c43b..f8c7711 100644 --- ql/src/test/results/clientpositive/spark/multi_insert.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert.q.out @@ -604,7 +604,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -766,7 +766,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -928,7 +928,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1090,7 +1090,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out index 6e5c3a9..b10c790 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out @@ -44,7 +44,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -239,7 +239,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out index 5b98d7f..67ff1ba 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out @@ -42,7 +42,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out index 4afb763..e054664 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out @@ -48,8 +48,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -174,8 +174,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1614,8 +1614,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1755,8 +1755,8 @@ STAGE PLANS: Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out index aba1ed8..ef0d3ef 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out @@ -568,7 +568,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) + Reducer 3 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -857,9 +857,9 @@ STAGE PLANS: Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) - Reducer 4 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1221,9 +1221,9 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Map 1 (GROUP SORT, 1) - Reducer 4 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out index c2bc349..dbf3a00 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out @@ -55,9 +55,9 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) Reducer 4 <- Map 1 (GROUP, 1) - Reducer 5 <- Reducer 4 (GROUP SORT, 1) + Reducer 5 <- Reducer 4 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out index 19f6d86..db158eb 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out @@ -604,7 +604,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -766,7 +766,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -928,7 +928,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1090,7 +1090,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2417,9 +2417,9 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2551,9 +2551,9 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2685,9 +2685,9 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2819,9 +2819,9 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2961,9 +2961,9 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3206,9 +3206,9 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3451,9 +3451,9 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3696,9 +3696,9 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/multi_join_union.q.out ql/src/test/results/clientpositive/spark/multi_join_union.q.out index 3f948a9..cfc354f 100644 --- ql/src/test/results/clientpositive/spark/multi_join_union.q.out +++ ql/src/test/results/clientpositive/spark/multi_join_union.q.out @@ -56,8 +56,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Union 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Union 5 (PARTITION-LEVEL SORT, 1) Union 5 <- Map 4 (NONE, 0), Map 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out index 2d0c4d7..475f624 100644 --- ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out +++ ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out @@ -459,7 +459,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out index b57d42a..41c81ed 100644 --- ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out +++ ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out @@ -221,7 +221,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1098,7 +1098,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1535,7 +1535,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Union 3 <- Map 4 (NONE, 0), Reducer 2 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -1792,7 +1792,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/order.q.out ql/src/test/results/clientpositive/spark/order.q.out index 3c5b169..b0abedf 100644 --- ql/src/test/results/clientpositive/spark/order.q.out +++ ql/src/test/results/clientpositive/spark/order.q.out @@ -12,7 +12,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -84,7 +84,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/order2.q.out ql/src/test/results/clientpositive/spark/order2.q.out index 8999399..6050e4a 100644 --- ql/src/test/results/clientpositive/spark/order2.q.out +++ ql/src/test/results/clientpositive/spark/order2.q.out @@ -16,7 +16,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/outer_join_ppr.q.out ql/src/test/results/clientpositive/spark/outer_join_ppr.q.out index 2645554..97b6c6c 100644 --- ql/src/test/results/clientpositive/spark/outer_join_ppr.q.out +++ ql/src/test/results/clientpositive/spark/outer_join_ppr.q.out @@ -108,7 +108,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -584,7 +584,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/parallel.q.out ql/src/test/results/clientpositive/spark/parallel.q.out index 640f557..62f8da0 100644 --- ql/src/test/results/clientpositive/spark/parallel.q.out +++ ql/src/test/results/clientpositive/spark/parallel.q.out @@ -41,7 +41,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/parallel_join0.q.out ql/src/test/results/clientpositive/spark/parallel_join0.q.out index 46a93cc..2231b38 100644 --- ql/src/test/results/clientpositive/spark/parallel_join0.q.out +++ ql/src/test/results/clientpositive/spark/parallel_join0.q.out @@ -26,8 +26,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 4) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 4) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/parallel_join1.q.out ql/src/test/results/clientpositive/spark/parallel_join1.q.out index 04730bd..6740c7f 100644 --- ql/src/test/results/clientpositive/spark/parallel_join1.q.out +++ ql/src/test/results/clientpositive/spark/parallel_join1.q.out @@ -28,7 +28,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 4), Map 3 (GROUP PARTITION-LEVEL SORT, 4) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/parquet_join.q.out ql/src/test/results/clientpositive/spark/parquet_join.q.out index f379624..4b9006f 100644 --- ql/src/test/results/clientpositive/spark/parquet_join.q.out +++ ql/src/test/results/clientpositive/spark/parquet_join.q.out @@ -72,7 +72,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -166,7 +166,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/pcr.q.out ql/src/test/results/clientpositive/spark/pcr.q.out index 1387948..86dc015 100644 --- ql/src/test/results/clientpositive/spark/pcr.q.out +++ ql/src/test/results/clientpositive/spark/pcr.q.out @@ -100,7 +100,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -323,7 +323,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -640,7 +640,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -881,7 +881,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1124,7 +1124,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1423,7 +1423,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1722,7 +1722,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1937,7 +1937,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2188,7 +2188,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2523,7 +2523,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2753,8 +2753,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3061,8 +3061,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3388,7 +3388,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3757,7 +3757,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -4630,7 +4630,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -4815,7 +4815,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -5044,7 +5044,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_gby_join.q.out ql/src/test/results/clientpositive/spark/ppd_gby_join.q.out index 1b6acee..1a78199 100644 --- ql/src/test/results/clientpositive/spark/ppd_gby_join.q.out +++ ql/src/test/results/clientpositive/spark/ppd_gby_join.q.out @@ -26,7 +26,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -154,7 +154,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/ppd_join.q.out ql/src/test/results/clientpositive/spark/ppd_join.q.out index 34be421..25cd98e 100644 --- ql/src/test/results/clientpositive/spark/ppd_join.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -564,7 +564,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_join2.q.out ql/src/test/results/clientpositive/spark/ppd_join2.q.out index d5e2b14..b98bc39 100644 --- ql/src/test/results/clientpositive/spark/ppd_join2.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join2.q.out @@ -30,8 +30,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1723,8 +1723,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_join3.q.out ql/src/test/results/clientpositive/spark/ppd_join3.q.out index b899967..507089d 100644 --- ql/src/test/results/clientpositive/spark/ppd_join3.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join3.q.out @@ -30,7 +30,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1761,7 +1761,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_join4.q.out ql/src/test/results/clientpositive/spark/ppd_join4.q.out index 3118508..04e4758 100644 --- ql/src/test/results/clientpositive/spark/ppd_join4.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join4.q.out @@ -52,8 +52,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_join5.q.out ql/src/test/results/clientpositive/spark/ppd_join5.q.out index 0886360..54b6b34 100644 --- ql/src/test/results/clientpositive/spark/ppd_join5.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join5.q.out @@ -50,8 +50,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -157,8 +157,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out index 66c616d..6ee0c8f 100644 --- ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out @@ -127,7 +127,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -505,7 +505,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -883,7 +883,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1257,7 +1257,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out index d98a9ff..6b952f9 100644 --- ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out +++ ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out @@ -51,7 +51,7 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1312,7 +1312,7 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out index 76b541d..ab31dfd 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -138,7 +138,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out index 6a66c83..96f14ef 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -258,7 +258,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out index d783562..0978c4f 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -252,7 +252,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out index e235fca..d29498a 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out @@ -30,7 +30,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -395,7 +395,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out index 1316b8a..e71c7f2 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out @@ -42,7 +42,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -128,7 +128,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -213,7 +213,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/ppd_transform.q.out ql/src/test/results/clientpositive/spark/ppd_transform.q.out index 282277a..1dfd4e9 100644 --- ql/src/test/results/clientpositive/spark/ppd_transform.q.out +++ ql/src/test/results/clientpositive/spark/ppd_transform.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -206,7 +206,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out index 0a3c821..54fe587 100644 --- ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out +++ ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out @@ -10,8 +10,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/router_join_ppr.q.out ql/src/test/results/clientpositive/spark/router_join_ppr.q.out index 84171c2..26fbe79 100644 --- ql/src/test/results/clientpositive/spark/router_join_ppr.q.out +++ ql/src/test/results/clientpositive/spark/router_join_ppr.q.out @@ -108,7 +108,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -592,7 +592,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -973,7 +973,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1354,7 +1354,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/sample10.q.out ql/src/test/results/clientpositive/spark/sample10.q.out index b456529..a9574ee 100644 --- ql/src/test/results/clientpositive/spark/sample10.q.out +++ ql/src/test/results/clientpositive/spark/sample10.q.out @@ -90,7 +90,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/sample6.q.out ql/src/test/results/clientpositive/spark/sample6.q.out index 8445610..f7ddbd0 100644 --- ql/src/test/results/clientpositive/spark/sample6.q.out +++ ql/src/test/results/clientpositive/spark/sample6.q.out @@ -487,7 +487,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -890,7 +890,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1547,7 +1547,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2047,7 +2047,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2533,7 +2533,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2866,7 +2866,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3076,7 +3076,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/sample8.q.out ql/src/test/results/clientpositive/spark/sample8.q.out index 3ac7356..17e3204 100644 --- ql/src/test/results/clientpositive/spark/sample8.q.out +++ ql/src/test/results/clientpositive/spark/sample8.q.out @@ -92,7 +92,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -807,7 +807,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -898,7 +898,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/semijoin.q.out ql/src/test/results/clientpositive/spark/semijoin.q.out index 9b6802d..d802069 100644 --- ql/src/test/results/clientpositive/spark/semijoin.q.out +++ ql/src/test/results/clientpositive/spark/semijoin.q.out @@ -126,8 +126,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -232,8 +232,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -340,8 +340,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -440,8 +440,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -551,8 +551,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -654,8 +654,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -757,8 +757,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -857,8 +857,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -962,8 +962,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1080,8 +1080,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1184,8 +1184,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1317,8 +1317,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1430,8 +1430,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1569,8 +1569,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1702,8 +1702,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1838,8 +1838,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1974,8 +1974,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2110,8 +2110,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2248,8 +2248,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2397,9 +2397,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2561,7 +2561,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoin.q.out ql/src/test/results/clientpositive/spark/skewjoin.q.out index c92415c..23d4b18 100644 --- ql/src/test/results/clientpositive/spark/skewjoin.q.out +++ ql/src/test/results/clientpositive/spark/skewjoin.q.out @@ -88,7 +88,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -198,7 +198,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -330,7 +330,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -454,7 +454,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -567,7 +567,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -695,7 +695,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -831,7 +831,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -976,7 +976,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out index 9cd9990..5d64049 100644 --- ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out +++ ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out @@ -15,8 +15,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoin_union_remove_1.q.out ql/src/test/results/clientpositive/spark/skewjoin_union_remove_1.q.out index a7ab07d..c4a58c1 100644 --- ql/src/test/results/clientpositive/spark/skewjoin_union_remove_1.q.out +++ ql/src/test/results/clientpositive/spark/skewjoin_union_remove_1.q.out @@ -68,8 +68,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -214,8 +214,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -369,8 +369,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -537,8 +537,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoin_union_remove_2.q.out ql/src/test/results/clientpositive/spark/skewjoin_union_remove_2.q.out index 2f1baee..a0fb663 100644 --- ql/src/test/results/clientpositive/spark/skewjoin_union_remove_2.q.out +++ ql/src/test/results/clientpositive/spark/skewjoin_union_remove_2.q.out @@ -80,8 +80,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1), Map 9 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1), Map 9 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out index 783ce82..d53b073 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out @@ -54,8 +54,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -200,8 +200,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -348,9 +348,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Union 3 (GROUP, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -494,9 +494,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Union 3 (GROUP, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out index 9e76fb6..c68f954 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out @@ -56,8 +56,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 5 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt11.q.out ql/src/test/results/clientpositive/spark/skewjoinopt11.q.out index e40def5..0b7fd10 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt11.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt11.q.out @@ -66,10 +66,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 11 <- Map 10 (GROUP PARTITION-LEVEL SORT, 1), Map 13 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 9 <- Map 12 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 11 <- Map 10 (PARTITION-LEVEL SORT, 1), Map 13 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 9 <- Map 12 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 11 (NONE, 0), Reducer 2 (NONE, 0), Reducer 6 (NONE, 0), Reducer 9 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt12.q.out ql/src/test/results/clientpositive/spark/skewjoinopt12.q.out index 99b91e4..15ff759 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt12.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt12.q.out @@ -56,8 +56,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt13.q.out ql/src/test/results/clientpositive/spark/skewjoinopt13.q.out index c002d5e..c4b294b 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt13.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt13.q.out @@ -82,8 +82,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt14.q.out ql/src/test/results/clientpositive/spark/skewjoinopt14.q.out index a380750..9c955bf 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt14.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt14.q.out @@ -86,9 +86,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 8 (GROUP PARTITION-LEVEL SORT, 1), Union 3 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 9 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 8 (PARTITION-LEVEL SORT, 1), Union 3 (PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 9 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out index f5ab51c..b1a0ee8 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out @@ -94,8 +94,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -240,8 +240,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -388,9 +388,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Union 3 (GROUP, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -534,9 +534,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Union 3 (GROUP, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt16.q.out ql/src/test/results/clientpositive/spark/skewjoinopt16.q.out index a48a560..07c17f4 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt16.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt16.q.out @@ -56,8 +56,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out index edd4d71..e4e7169 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out @@ -60,8 +60,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -262,8 +262,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out index 3f237ec..708e304 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out @@ -82,7 +82,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt19.q.out ql/src/test/results/clientpositive/spark/skewjoinopt19.q.out index a3ce3d8..d599cd5 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt19.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt19.q.out @@ -58,8 +58,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out index 31213df..ac1d6c6 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out @@ -64,8 +64,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -203,8 +203,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -346,9 +346,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Union 3 (GROUP, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -506,9 +506,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Union 3 (GROUP, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt20.q.out ql/src/test/results/clientpositive/spark/skewjoinopt20.q.out index 39fcd7d..2ed3634 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt20.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt20.q.out @@ -58,8 +58,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out index b80c1ee..93cdbb8 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out @@ -58,8 +58,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -204,8 +204,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out index ca5c82a..1df961d 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out @@ -54,8 +54,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -198,8 +198,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt5.q.out ql/src/test/results/clientpositive/spark/skewjoinopt5.q.out index 228a898..7712c0e 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt5.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt5.q.out @@ -56,8 +56,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt6.q.out ql/src/test/results/clientpositive/spark/skewjoinopt6.q.out index e8dc4ef..bdab7de 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt6.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt6.q.out @@ -58,8 +58,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 6 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt7.q.out ql/src/test/results/clientpositive/spark/skewjoinopt7.q.out index aedb192..3d46cf2 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt7.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt7.q.out @@ -74,8 +74,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1), Map 9 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1), Map 9 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt8.q.out ql/src/test/results/clientpositive/spark/skewjoinopt8.q.out index ebd350f..76c674c 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt8.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt8.q.out @@ -72,8 +72,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1), Map 9 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1), Map 9 (PARTITION-LEVEL SORT, 1) Union 3 <- Reducer 2 (NONE, 0), Reducer 7 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out index 8188487..5e2da76 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out @@ -64,7 +64,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Union 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Union 4 (PARTITION-LEVEL SORT, 1) Union 4 <- Map 3 (NONE, 0), Map 5 (NONE, 0) #### A masked pattern was here #### Vertices: @@ -203,7 +203,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin9.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin9.q.out index d21a9eb..cadf7ea 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin9.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin9.q.out @@ -113,7 +113,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -274,7 +274,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -383,7 +383,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_1.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_1.q.out index bfde76c..d58d226 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_1.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_1.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -143,7 +143,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -225,7 +225,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -306,7 +306,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -392,7 +392,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -475,7 +475,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -557,7 +557,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -638,7 +638,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_10.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_10.q.out index 747150b..6b62fdc 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_10.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_10.q.out @@ -80,7 +80,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out index 8dffd07..23835ba 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out @@ -128,8 +128,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -416,8 +416,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out index c297224..9b6ea3c 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out @@ -58,7 +58,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -176,9 +176,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 4 <- Reducer 3 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -326,7 +326,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### @@ -478,7 +478,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -616,7 +616,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -774,7 +774,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -926,7 +926,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1054,7 +1054,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1178,7 +1178,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1294,7 +1294,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1420,7 +1420,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1585,7 +1585,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out index b50a1ee..46f4ee7 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out @@ -100,8 +100,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -450,8 +450,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -748,8 +748,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1046,8 +1046,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out index e1f4fc5..f56422d 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out @@ -50,7 +50,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out index c27c399..7a26fdc 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out @@ -188,7 +188,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1), Map 9 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1), Map 9 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -682,7 +682,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_2.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_2.q.out index b7e6715..9d3000e 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_2.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_2.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -145,7 +145,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -227,7 +227,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -310,7 +310,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -396,7 +396,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -481,7 +481,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -563,7 +563,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -646,7 +646,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out index 99a25bf..4cf79d7 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out @@ -54,7 +54,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -312,7 +312,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out index 5bdda20..fbee50a 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out @@ -131,7 +131,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -220,7 +220,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -309,7 +309,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -398,7 +398,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out index efd3f1b..4fda89b 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out @@ -60,9 +60,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 6 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -204,7 +204,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_3.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_3.q.out index ebe22b4..58aefee 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_3.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_3.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -145,7 +145,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -226,7 +226,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -309,7 +309,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -394,7 +394,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -479,7 +479,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -560,7 +560,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -643,7 +643,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_4.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_4.q.out index c5ff5b8..893e4cf 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_4.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_4.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -161,7 +161,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -253,7 +253,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -350,7 +350,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -448,7 +448,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -551,7 +551,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -645,7 +645,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -741,7 +741,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -839,7 +839,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -939,7 +939,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1033,7 +1033,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1134,7 +1134,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1232,7 +1232,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_5.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_5.q.out index e82038b..01e11a5 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_5.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_5.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -161,7 +161,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -253,7 +253,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -350,7 +350,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -448,7 +448,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -551,7 +551,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -645,7 +645,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -741,7 +741,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -839,7 +839,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -939,7 +939,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1033,7 +1033,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1134,7 +1134,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1232,7 +1232,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out index f85e34a..63a16b6 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out @@ -72,7 +72,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1244,7 +1244,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2432,7 +2432,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2536,7 +2536,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2636,7 +2636,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_7.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_7.q.out index 1e003f3..bdb36d0 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_7.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_7.q.out @@ -624,7 +624,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/sort.q.out ql/src/test/results/clientpositive/spark/sort.q.out index 04f6c32..7e33d8a 100644 --- ql/src/test/results/clientpositive/spark/sort.q.out +++ ql/src/test/results/clientpositive/spark/sort.q.out @@ -12,7 +12,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out index 5781913..78bf282 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out @@ -62,7 +62,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out index d88c6f0..9032570 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out @@ -70,7 +70,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out index 51da7c5..d2504f1 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out @@ -70,7 +70,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out index f52cc6d..61e1b1b 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out @@ -68,7 +68,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out index 4b98029..f326f78 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out @@ -119,7 +119,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out index 9e66c46..cd2f225 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out @@ -119,7 +119,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out index dd46fd9..feca353 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out @@ -155,7 +155,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out index 6aa6f00..a9dc3c9 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out @@ -115,7 +115,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -223,7 +223,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/subquery_exists.q.out ql/src/test/results/clientpositive/spark/subquery_exists.q.out index 8244da5..5956bf3 100644 --- ql/src/test/results/clientpositive/spark/subquery_exists.q.out +++ ql/src/test/results/clientpositive/spark/subquery_exists.q.out @@ -26,7 +26,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/subquery_in.q.out ql/src/test/results/clientpositive/spark/subquery_in.q.out index 29e2fd6..c56b3ff 100644 --- ql/src/test/results/clientpositive/spark/subquery_in.q.out +++ ql/src/test/results/clientpositive/spark/subquery_in.q.out @@ -118,7 +118,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -236,7 +236,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -361,9 +361,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) - Reducer 4 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -513,8 +513,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -673,7 +673,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -836,8 +836,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out index 38ae609..85939c0 100644 --- ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out +++ ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out @@ -68,10 +68,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 9 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 7 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) - Reducer 5 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (SORT, 1) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) Reducer 9 <- Map 8 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -501,10 +501,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 9 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 7 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) - Reducer 5 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (SORT, 1) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) Reducer 9 <- Map 8 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/temp_table.q.out ql/src/test/results/clientpositive/spark/temp_table.q.out index 7215057..1ccc56c 100644 --- ql/src/test/results/clientpositive/spark/temp_table.q.out +++ ql/src/test/results/clientpositive/spark/temp_table.q.out @@ -208,7 +208,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Union 2 (GROUP SORT, 1) + Reducer 3 <- Union 2 (SORT, 1) Union 2 <- Map 1 (NONE, 0), Map 4 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/temp_table_join1.q.out ql/src/test/results/clientpositive/spark/temp_table_join1.q.out index 7e78ed5..81bceb2 100644 --- ql/src/test/results/clientpositive/spark/temp_table_join1.q.out +++ ql/src/test/results/clientpositive/spark/temp_table_join1.q.out @@ -36,7 +36,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -132,7 +132,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -230,7 +230,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/tez_join_tests.q.out ql/src/test/results/clientpositive/spark/tez_join_tests.q.out index 9254944..14406df 100644 --- ql/src/test/results/clientpositive/spark/tez_join_tests.q.out +++ ql/src/test/results/clientpositive/spark/tez_join_tests.q.out @@ -12,10 +12,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 3 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Reducer 4 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/tez_joins_explain.q.out ql/src/test/results/clientpositive/spark/tez_joins_explain.q.out index d2b23ad..3f080a6 100644 --- ql/src/test/results/clientpositive/spark/tez_joins_explain.q.out +++ ql/src/test/results/clientpositive/spark/tez_joins_explain.q.out @@ -12,10 +12,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 3 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 5 <- Reducer 4 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/transform_ppr1.q.out ql/src/test/results/clientpositive/spark/transform_ppr1.q.out index 70bf0fd..d55e9c4 100644 --- ql/src/test/results/clientpositive/spark/transform_ppr1.q.out +++ ql/src/test/results/clientpositive/spark/transform_ppr1.q.out @@ -99,7 +99,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/transform_ppr2.q.out ql/src/test/results/clientpositive/spark/transform_ppr2.q.out index ea7ec5a..38cfd8b 100644 --- ql/src/test/results/clientpositive/spark/transform_ppr2.q.out +++ ql/src/test/results/clientpositive/spark/transform_ppr2.q.out @@ -101,7 +101,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/union23.q.out ql/src/test/results/clientpositive/spark/union23.q.out index 9e26762..e61b324 100644 --- ql/src/test/results/clientpositive/spark/union23.q.out +++ ql/src/test/results/clientpositive/spark/union23.q.out @@ -24,7 +24,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Union 2 (GROUP SORT, 1) + Reducer 3 <- Union 2 (SORT, 1) Union 2 <- Map 1 (NONE, 0), Map 4 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/union3.q.out ql/src/test/results/clientpositive/spark/union3.q.out index 7782965..f835d8c 100644 --- ql/src/test/results/clientpositive/spark/union3.q.out +++ ql/src/test/results/clientpositive/spark/union3.q.out @@ -47,11 +47,11 @@ STAGE PLANS: Spark Edges: Reducer 10 <- Map 9 (GROUP, 1) - Reducer 11 <- Reducer 10 (GROUP SORT, 1) + Reducer 11 <- Reducer 10 (SORT, 1) Reducer 2 <- Map 1 (GROUP, 1) Reducer 5 <- Map 4 (GROUP, 1) Reducer 7 <- Map 6 (GROUP, 1) - Reducer 8 <- Reducer 7 (GROUP SORT, 1) + Reducer 8 <- Reducer 7 (SORT, 1) Union 3 <- Reducer 11 (NONE, 0), Reducer 2 (NONE, 0), Reducer 5 (NONE, 0), Reducer 8 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/union33.q.out ql/src/test/results/clientpositive/spark/union33.q.out index 7ac8dea..dc6ab25 100644 --- ql/src/test/results/clientpositive/spark/union33.q.out +++ ql/src/test/results/clientpositive/spark/union33.q.out @@ -42,7 +42,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 1) Reducer 5 <- Reducer 4 (GROUP, 1) Union 2 <- Map 1 (NONE, 0), Reducer 5 (NONE, 0) #### A masked pattern was here #### @@ -201,7 +201,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) Union 4 <- Map 5 (NONE, 0), Reducer 3 (NONE, 0) #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/union_ppr.q.out ql/src/test/results/clientpositive/spark/union_ppr.q.out index 24b6912..7611f3b 100644 --- ql/src/test/results/clientpositive/spark/union_ppr.q.out +++ ql/src/test/results/clientpositive/spark/union_ppr.q.out @@ -112,7 +112,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Union 2 (GROUP SORT, 1) + Reducer 3 <- Union 2 (SORT, 1) Union 2 <- Map 1 (NONE, 0), Map 4 (NONE, 0) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/vector_between_in.q.out ql/src/test/results/clientpositive/spark/vector_between_in.q.out index 7d1323b..f214435 100644 --- ql/src/test/results/clientpositive/spark/vector_between_in.q.out +++ ql/src/test/results/clientpositive/spark/vector_between_in.q.out @@ -20,7 +20,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -134,7 +134,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -248,7 +248,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -301,7 +301,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -354,7 +354,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out index 60ab429..5ff382f 100644 --- ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out +++ ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out @@ -120,7 +120,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out index 19d45e1..f042c4a 100644 --- ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out +++ ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out @@ -1247,7 +1247,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_data_types.q.out ql/src/test/results/clientpositive/spark/vector_data_types.q.out index 12a5f95..05f0f54 100644 --- ql/src/test/results/clientpositive/spark/vector_data_types.q.out +++ ql/src/test/results/clientpositive/spark/vector_data_types.q.out @@ -109,7 +109,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -200,7 +200,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out index e5e24c8..a578237 100644 --- ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out +++ ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out @@ -38,7 +38,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_left_outer_join.q.out ql/src/test/results/clientpositive/spark/vector_left_outer_join.q.out index 31e2cd7..d505232 100644 --- ql/src/test/results/clientpositive/spark/vector_left_outer_join.q.out +++ ql/src/test/results/clientpositive/spark/vector_left_outer_join.q.out @@ -24,9 +24,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) - Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out index 44ae1d3..b4eaedc 100644 --- ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out @@ -126,8 +126,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -296,8 +296,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out index 8122e10..6a7c990 100644 --- ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out @@ -116,7 +116,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorization_0.q.out ql/src/test/results/clientpositive/spark/vectorization_0.q.out index 82e4926..56d300f 100644 --- ql/src/test/results/clientpositive/spark/vectorization_0.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_0.q.out @@ -25,7 +25,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -123,7 +123,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -230,7 +230,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -343,7 +343,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -441,7 +441,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -548,7 +548,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -661,7 +661,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -759,7 +759,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -866,7 +866,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorization_14.q.out ql/src/test/results/clientpositive/spark/vectorization_14.q.out index 2d49a0c..78d50d2 100644 --- ql/src/test/results/clientpositive/spark/vectorization_14.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_14.q.out @@ -73,7 +73,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorization_15.q.out ql/src/test/results/clientpositive/spark/vectorization_15.q.out index f9f4476..caa677e 100644 --- ql/src/test/results/clientpositive/spark/vectorization_15.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_15.q.out @@ -69,7 +69,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorization_div0.q.out ql/src/test/results/clientpositive/spark/vectorization_div0.q.out index 6f19862..30d116f 100644 --- ql/src/test/results/clientpositive/spark/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_div0.q.out @@ -156,7 +156,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -331,7 +331,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out index 5f4f3bb..0f10727 100644 --- ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out @@ -58,7 +58,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out index 3722e76..d02c96e 100644 --- ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out @@ -926,7 +926,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1185,7 +1185,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1393,7 +1393,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1659,7 +1659,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1867,7 +1867,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2075,7 +2075,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2330,7 +2330,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2663,7 +2663,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out index 354a7ff..882ce5f 100644 --- ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out @@ -108,7 +108,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -197,7 +197,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -295,7 +295,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorized_mapjoin.q.out ql/src/test/results/clientpositive/spark/vectorized_mapjoin.q.out index 5b9205b..85fa250 100644 --- ql/src/test/results/clientpositive/spark/vectorized_mapjoin.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_mapjoin.q.out @@ -14,7 +14,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out index 1c226fd..4d4dd94 100644 --- ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out @@ -10,9 +10,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) - Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out index 577df70..d1a4c4f 100644 --- ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out @@ -224,8 +224,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -509,9 +509,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -831,7 +831,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1087,8 +1087,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1375,8 +1375,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1672,9 +1672,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1937,8 +1937,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2250,8 +2250,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2572,8 +2572,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -2849,8 +2849,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3130,8 +3130,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3418,9 +3418,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -3745,8 +3745,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -4130,9 +4130,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 5 <- Map 4 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -4457,7 +4457,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -4729,8 +4729,8 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -5202,10 +5202,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) - Reducer 5 <- Reducer 2 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -5711,9 +5711,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -6067,10 +6067,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) - Reducer 5 <- Reducer 4 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -6421,9 +6421,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -6770,10 +6770,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) - Reducer 5 <- Reducer 4 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -7169,9 +7169,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -7518,9 +7518,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) - Reducer 3 <- Reducer 2 (GROUP SORT, 1) - Reducer 4 <- Reducer 3 (GROUP SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out index 5b9205b..85fa250 100644 --- ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out @@ -14,7 +14,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out index b01d9a3..85da980 100644 --- ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out @@ -105,7 +105,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -248,7 +248,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -391,7 +391,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -538,7 +538,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP SORT, 1) + Reducer 2 <- Map 1 (SORT, 1) #### A masked pattern was here #### Vertices: Map 1