diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java index a456d6c..b6dd8cb 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java @@ -42,4 +42,9 @@ * Get the count of executors */ public int getExecutorCount() throws Exception; + + /** + * Get default parallelism. For standalone mode, this can be used to get total number of cores. + */ + public int getDefaultParallelism() throws Exception; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java index 78c1da8..d0abec8 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java @@ -62,7 +62,6 @@ public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf) sparkConf.put("spark.app.name", SPARK_DEFAULT_APP_NAME); sparkConf.put("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - sparkConf.put("spark.default.parallelism", "1"); // load properties from spark-defaults.conf. InputStream inputStream = null; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java index 5cfdcec..467147c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java @@ -95,6 +95,11 @@ public int getExecutorCount() { } @Override + public int getDefaultParallelism() throws Exception { + return sc.sc().defaultParallelism(); + } + + @Override public SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception { Context ctx = driverContext.getCtx(); HiveConf hiveConf = (HiveConf) ctx.getConf(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index 87b47a6..59003c4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -95,6 +95,14 @@ public int getExecutorCount() throws Exception { } @Override + public int getDefaultParallelism() throws Exception { + long timeout = hiveConf.getTimeVar( + HiveConf.ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT, TimeUnit.SECONDS); + Future handler = remoteClient.getDefaultParallelism(); + return handler.get(timeout, TimeUnit.SECONDS); + } + + @Override public SparkJobRef execute(final DriverContext driverContext, final SparkWork sparkWork) throws Exception { final Context ctx = driverContext.getCtx(); final HiveConf hiveConf = (HiveConf) ctx.getConf(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java index 12e0e28..b894c8b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java @@ -68,19 +68,34 @@ public SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) thro @Override public Tuple2 getMemoryAndCores() throws Exception { SparkConf sparkConf = hiveSparkClient.getSparkConf(); - int cores = sparkConf.getInt("spark.executor.cores", 1); - double memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.2); + int numExecutors = hiveSparkClient.getExecutorCount(); + // at start-up, we may be unable to get number of executors + if (numExecutors <= 0) { + return new Tuple2(-1L, -1); + } int executorMemoryInMB = Utils.memoryStringToMb( - sparkConf.get("spark.executor.memory", "512m")); - long memoryPerTaskInBytes = - (long) (executorMemoryInMB * memoryFraction * 1024 * 1024 / cores); - int executors = hiveSparkClient.getExecutorCount(); - int totalCores = executors * cores; - LOG.info("Spark cluster current has executors: " + executors - + ", cores per executor: " + cores + ", memory per executor: " - + executorMemoryInMB + "M, shuffle memoryFraction: " + memoryFraction); + sparkConf.get("spark.executor.memory", "512m")); + double memoryFraction = 1.0 - sparkConf.getDouble("spark.storage.memoryFraction", 0.6); + long totalMemory = (long) (numExecutors * executorMemoryInMB * memoryFraction * 1024 * 1024); + int totalCores; + String masterURL = sparkConf.get("spark.master"); + if (masterURL.startsWith("spark")) { + totalCores = sparkConf.contains("spark.default.parallelism") ? + sparkConf.getInt("spark.default.parallelism", 1) : + hiveSparkClient.getDefaultParallelism(); + totalCores = Math.max(totalCores, numExecutors); + } else { + int coresPerExecutor = sparkConf.getInt("spark.executor.cores", 1); + totalCores = numExecutors * coresPerExecutor; + } + totalCores = totalCores / sparkConf.getInt("spark.task.cpus", 1); + + long memoryPerTaskInBytes = totalMemory / totalCores; + LOG.info("Spark cluster current has executors: " + numExecutors + + ", total cores: " + totalCores + ", memory per executor: " + + executorMemoryInMB + "M, memoryFraction: " + memoryFraction); return new Tuple2(Long.valueOf(memoryPerTaskInBytes), - Integer.valueOf(totalCores)); + Integer.valueOf(totalCores)); } @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java index 613b5bc..b247fcf 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java @@ -117,7 +117,7 @@ public Object process(Node nd, Stack stack, context.getConf(), sparkSessionManager); sparkMemoryAndCores = sparkSession.getMemoryAndCores(); } catch (Exception e) { - throw new SemanticException("Failed to get spark memory/core info: " + e, e); + LOG.warn("Failed to get spark memory/core info", e); } finally { if (sparkSession != null && sparkSessionManager != null) { try { @@ -130,18 +130,24 @@ public Object process(Node nd, Stack stack, } // Divide it by 2 so that we can have more reducers - long bytesPerReducer = sparkMemoryAndCores._1.longValue() / 2; + long bytesPerReducer = context.getConf().getLongVar(HiveConf.ConfVars.BYTESPERREDUCER) / 2; int numReducers = Utilities.estimateReducers(numberOfBytes, bytesPerReducer, - maxReducers, false); + maxReducers, false); + + if (sparkMemoryAndCores != null && + sparkMemoryAndCores._1() > 0 && sparkMemoryAndCores._2() > 0) { + // warn the user if bytes per reducer is much larger than memory per task + if ((double) sparkMemoryAndCores._1() / bytesPerReducer < 0.5) { + LOG.warn("Average load of a reducer is much larger than its available memory. " + + "Consider decreasing hive.exec.reducers.bytes.per.reducer"); + } - // If there are more cores, use the number of cores - int cores = sparkMemoryAndCores._2.intValue(); - if (numReducers < cores) { - numReducers = cores; + // If there are more cores, use the number of cores + numReducers = Math.max(numReducers, sparkMemoryAndCores._2()); } - LOG.info("Set parallelism parameters: cores = " + cores + ", numReducers = " + numReducers + - ", bytesPerReducer = " + bytesPerReducer + ", numberOfBytes = " + numberOfBytes); - LOG.info("Set parallelism for reduce sink " + sink + " to: " + numReducers + " (calculated)"); + numReducers = Math.min(numReducers, maxReducers); + LOG.info("Set parallelism for reduce sink " + sink + " to: " + numReducers + + " (calculated)"); desc.setNumReducers(numReducers); } } else { 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 a184350..977bbcb 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 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -237,7 +237,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -310,7 +310,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -384,7 +384,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -461,7 +461,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -562,7 +562,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -665,7 +665,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### 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 2110bc7..8274fa7 100644 --- ql/src/test/results/clientpositive/spark/auto_join20.q.out +++ ql/src/test/results/clientpositive/spark/auto_join20.q.out @@ -229,7 +229,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 3) + Reducer 4 <- Map 3 (SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join29.q.out ql/src/test/results/clientpositive/spark/auto_join29.q.out index 78b6759..c5358dc 100644 --- ql/src/test/results/clientpositive/spark/auto_join29.q.out +++ ql/src/test/results/clientpositive/spark/auto_join29.q.out @@ -678,7 +678,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1299,7 +1299,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 3) + Reducer 3 <- Map 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 2 @@ -1929,7 +1929,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 3) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -2554,7 +2554,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2674,7 +2674,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 3) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -3295,7 +3295,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3416,7 +3416,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 3) + Reducer 3 <- Map 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 2 @@ -3554,7 +3554,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### 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 f0f2bce..213bb89 100644 --- ql/src/test/results/clientpositive/spark/auto_join30.q.out +++ ql/src/test/results/clientpositive/spark/auto_join30.q.out @@ -175,7 +175,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 3) + Reducer 5 <- Map 4 (SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -208,7 +208,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -321,7 +321,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -353,7 +353,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 3) + Reducer 4 <- Map 3 (SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -473,8 +473,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 3) - Reducer 7 <- Map 6 (SORT, 3) + Reducer 5 <- Map 4 (SORT, 2) + Reducer 7 <- Map 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -541,7 +541,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -672,8 +672,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 3) - Reducer 7 <- Map 6 (SORT, 3) + Reducer 5 <- Map 4 (SORT, 2) + Reducer 7 <- Map 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -734,7 +734,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -862,8 +862,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 3) - Reducer 7 <- Map 6 (SORT, 3) + Reducer 5 <- Map 4 (SORT, 2) + Reducer 7 <- Map 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -924,7 +924,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1052,8 +1052,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) - Reducer 4 <- Map 3 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1113,7 +1113,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (SORT, 3) + Reducer 6 <- Map 5 (SORT, 2) Reducer 7 <- Reducer 6 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1242,8 +1242,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) - Reducer 4 <- Map 3 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1303,7 +1303,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (SORT, 3) + Reducer 6 <- Map 5 (SORT, 2) Reducer 7 <- Reducer 6 (GROUP, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/auto_join32.q.out ql/src/test/results/clientpositive/spark/auto_join32.q.out index 0224acc..6f55200 100644 --- ql/src/test/results/clientpositive/spark/auto_join32.q.out +++ ql/src/test/results/clientpositive/spark/auto_join32.q.out @@ -166,7 +166,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -288,7 +288,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -434,7 +434,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### 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 2ff4ac2..63d2799 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 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Map 8 (PARTITION-LEVEL SORT, 3), Reducer 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 9 (PARTITION-LEVEL SORT, 3), Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 8 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 9 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/auto_join_stats.q.out ql/src/test/results/clientpositive/spark/auto_join_stats.q.out index 1c33e21..1fa1a74 100644 --- ql/src/test/results/clientpositive/spark/auto_join_stats.q.out +++ ql/src/test/results/clientpositive/spark/auto_join_stats.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -214,7 +214,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 daf6d11..56bbaf9 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 @@ -143,8 +143,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -316,8 +316,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 5 (PARTITION-LEVEL SORT, 3) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: 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 ed4bb77..23f19c7 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 @@ -159,7 +159,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -306,9 +306,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 5 <- Map 4 (GROUP, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3), Reducer 5 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP, 1) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -965,7 +965,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + 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: @@ -1641,7 +1641,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 1 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 d8b9c31..fe7b96d 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 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + 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: @@ -222,7 +222,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 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 ec363aa..3924c58 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 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + 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: @@ -208,7 +208,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + 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: @@ -320,7 +320,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + 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: @@ -428,7 +428,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + 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: @@ -624,7 +624,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + 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: @@ -732,7 +732,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + 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: @@ -840,7 +840,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + 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: @@ -1036,7 +1036,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + 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 e7dd475..a2b98fc 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 @@ -153,7 +153,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -264,7 +264,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -412,7 +412,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 4 <- Map 3 (GROUP, 3) + Reducer 4 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 3 @@ -461,7 +461,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1756,7 +1756,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -1891,7 +1891,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -2064,7 +2064,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (GROUP, 3) + Reducer 5 <- Map 4 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 4 @@ -2136,7 +2136,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 1 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 09b180b..1c81d1b 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 @@ -206,7 +206,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -348,7 +348,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -464,7 +464,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -776,7 +776,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -875,7 +875,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### 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 cea2138..04a934f 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 @@ -556,7 +556,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -646,7 +646,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### 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 30e6095..4ffd934 100644 --- ql/src/test/results/clientpositive/spark/column_access_stats.q.out +++ ql/src/test/results/clientpositive/spark/column_access_stats.q.out @@ -371,7 +371,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -494,7 +494,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -584,7 +584,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -696,8 +696,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### 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 3e4ac5c..fa746d8 100644 --- ql/src/test/results/clientpositive/spark/count.q.out +++ ql/src/test/results/clientpositive/spark/count.q.out @@ -186,7 +186,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/cross_product_check_1.q.out ql/src/test/results/clientpositive/spark/cross_product_check_1.q.out index de3f7ae..099a60f 100644 --- ql/src/test/results/clientpositive/spark/cross_product_check_1.q.out +++ ql/src/test/results/clientpositive/spark/cross_product_check_1.q.out @@ -99,7 +99,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: @@ -201,9 +201,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -316,7 +316,7 @@ STAGE PLANS: Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) Reducer 4 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -421,9 +421,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 7 (PARTITION-LEVEL SORT, 3) - Reducer 6 <- Reducer 5 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 2) Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/cross_product_check_2.q.out ql/src/test/results/clientpositive/spark/cross_product_check_2.q.out index f80ef91..dda6c38 100644 --- ql/src/test/results/clientpositive/spark/cross_product_check_2.q.out +++ ql/src/test/results/clientpositive/spark/cross_product_check_2.q.out @@ -229,7 +229,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -347,7 +347,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -468,7 +468,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 4 <- Map 3 (GROUP, 3) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -518,7 +518,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### 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 7a75599..bf7c0a4 100644 --- ql/src/test/results/clientpositive/spark/ctas.q.out +++ ql/src/test/results/clientpositive/spark/ctas.q.out @@ -176,7 +176,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -320,7 +320,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -529,7 +529,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -716,7 +716,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out index 60d45ad..286c1c4 100644 --- ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out +++ ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out @@ -142,8 +142,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby10.q.out ql/src/test/results/clientpositive/spark/groupby10.q.out index 31ea336..02bafe4 100644 --- ql/src/test/results/clientpositive/spark/groupby10.q.out +++ ql/src/test/results/clientpositive/spark/groupby10.q.out @@ -268,10 +268,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (GROUP, 3) - Reducer 4 <- Reducer 6 (GROUP, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (GROUP, 2) + Reducer 4 <- Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -483,10 +483,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (GROUP, 3) - Reducer 4 <- Reducer 6 (GROUP, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (GROUP, 2) + Reducer 4 <- Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby8.q.out ql/src/test/results/clientpositive/spark/groupby8.q.out index c0c0cf2..eb63264 100644 --- ql/src/test/results/clientpositive/spark/groupby8.q.out +++ ql/src/test/results/clientpositive/spark/groupby8.q.out @@ -828,10 +828,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (GROUP, 3) - Reducer 4 <- Reducer 6 (GROUP, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (GROUP, 2) + Reducer 4 <- Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby9.q.out ql/src/test/results/clientpositive/spark/groupby9.q.out index 2707239..fd2dfe6 100644 --- ql/src/test/results/clientpositive/spark/groupby9.q.out +++ ql/src/test/results/clientpositive/spark/groupby9.q.out @@ -829,10 +829,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (GROUP, 3) - Reducer 4 <- Reducer 6 (GROUP, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (GROUP, 2) + Reducer 4 <- Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1619,10 +1619,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (GROUP, 3) - Reducer 4 <- Reducer 6 (GROUP, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (GROUP, 2) + Reducer 4 <- Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2409,8 +2409,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP, 3) - Reducer 3 <- Map 5 (GROUP, 3) + Reducer 2 <- Map 4 (GROUP, 2) + Reducer 3 <- Map 5 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -3196,10 +3196,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (GROUP, 3) - Reducer 4 <- Reducer 6 (GROUP, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (GROUP, 2) + Reducer 4 <- Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/groupby_cube1.q.out ql/src/test/results/clientpositive/spark/groupby_cube1.q.out index 8ed3e67..27cd671 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 PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -199,8 +199,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -304,7 +304,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -404,10 +404,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 6 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 5 <- Reducer 4 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 6 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 6 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 fca42bb..d368807 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 @@ -224,8 +224,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 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 6229ceb..a157235 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 @@ -299,8 +299,8 @@ STAGE PLANS: Stage: Stage-5 Spark Edges: - Reducer 2 <- Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (SORT, 1) #### A masked pattern was here #### Vertices: 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 d32a360..5ea8d19 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 @@ -221,7 +221,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -393,7 +393,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -565,7 +565,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### 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 a75c52a..375470a 100644 --- ql/src/test/results/clientpositive/spark/groupby_position.q.out +++ ql/src/test/results/clientpositive/spark/groupby_position.q.out @@ -230,8 +230,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -426,7 +426,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -548,9 +548,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 6 <- Map 5 (GROUP, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3), Reducer 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (SORT, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out index 8905705..701a62a 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 PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -193,8 +193,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -292,7 +292,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -392,10 +392,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 6 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 5 <- Reducer 4 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 6 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 6 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 c838118..191ea49 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 @@ -297,7 +297,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1184,7 +1184,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1411,7 +1411,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1674,7 +1674,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1983,7 +1983,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -2082,7 +2082,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -2165,7 +2165,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -2309,7 +2309,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP, 3) + Reducer 3 <- Map 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2712,7 +2712,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2902,7 +2902,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 4 + numFiles 3 numRows 10 rawDataSize 32 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -2932,7 +2932,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 4 + numFiles 3 numRows 10 rawDataSize 32 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -3076,8 +3076,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (GROUP, 3) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 4 <- Map 3 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3360,7 +3360,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3472,7 +3472,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -3502,7 +3502,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -3635,7 +3635,7 @@ STAGE PLANS: columns.types int:int:string:int #### A masked pattern was here #### name default.outputtbl4 - numFiles 3 + numFiles 2 numRows 6 rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} @@ -3718,7 +3718,7 @@ STAGE PLANS: columns.types int:int:string:int #### A masked pattern was here #### name default.outputtbl4 - numFiles 3 + numFiles 2 numRows 6 rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} 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 39f53ed..0c1bac6 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 @@ -297,8 +297,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1202,8 +1202,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1447,8 +1447,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1728,8 +1728,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2055,7 +2055,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -2154,7 +2154,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -2237,7 +2237,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -2381,8 +2381,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (GROUP, 3) + Reducer 3 <- Map 2 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2802,7 +2802,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2992,7 +2992,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 4 + numFiles 3 numRows 10 rawDataSize 32 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -3022,7 +3022,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 4 + numFiles 3 numRows 10 rawDataSize 32 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -3166,9 +3166,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 5 <- Reducer 4 (GROUP, 3) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 5 (PARTITION-LEVEL SORT, 3) + Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3468,8 +3468,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3598,7 +3598,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -3628,7 +3628,7 @@ STAGE PLANS: columns.types int:int #### A masked pattern was here #### name default.outputtbl1 - numFiles 3 + numFiles 2 numRows 5 rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} @@ -3761,7 +3761,7 @@ STAGE PLANS: columns.types int:int:string:int #### A masked pattern was here #### name default.outputtbl4 - numFiles 3 + numFiles 2 numRows 6 rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} @@ -3844,7 +3844,7 @@ STAGE PLANS: columns.types int:int:string:int #### A masked pattern was here #### name default.outputtbl4 - numFiles 3 + numFiles 2 numRows 6 rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} diff --git ql/src/test/results/clientpositive/spark/having.q.out ql/src/test/results/clientpositive/spark/having.q.out index 2659275..a69cea5 100644 --- ql/src/test/results/clientpositive/spark/having.q.out +++ ql/src/test/results/clientpositive/spark/having.q.out @@ -95,7 +95,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -472,7 +472,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -744,7 +744,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -941,7 +941,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1213,7 +1213,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 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 7f19a01..af366a5 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 @@ -126,7 +126,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 d4abf5b..0c54d0e 100644 --- ql/src/test/results/clientpositive/spark/innerjoin.q.out +++ ql/src/test/results/clientpositive/spark/innerjoin.q.out @@ -1189,7 +1189,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 a0687d8..0b5f8fb 100644 --- ql/src/test/results/clientpositive/spark/join20.q.out +++ ql/src/test/results/clientpositive/spark/join20.q.out @@ -675,8 +675,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### 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 7a9ae71..b1ca0e5 100644 --- ql/src/test/results/clientpositive/spark/join40.q.out +++ ql/src/test/results/clientpositive/spark/join40.q.out @@ -666,7 +666,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1778,8 +1778,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2439,8 +2439,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3750,7 +3750,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) 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 aff5b0b..0878104 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 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -105,7 +105,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 750e24d..d63b4f8 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 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -459,7 +459,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -832,7 +832,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1219,7 +1219,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1664,7 +1664,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) #### 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 5db41bf..98c5c02 100644 --- ql/src/test/results/clientpositive/spark/join_rc.q.out +++ ql/src/test/results/clientpositive/spark/join_rc.q.out @@ -54,7 +54,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 a8f2e29..9f075c0 100644 --- ql/src/test/results/clientpositive/spark/join_reorder.q.out +++ ql/src/test/results/clientpositive/spark/join_reorder.q.out @@ -246,8 +246,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -343,8 +343,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -486,7 +486,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -570,7 +570,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### 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 5a55428..2555451 100644 --- ql/src/test/results/clientpositive/spark/join_reorder2.q.out +++ ql/src/test/results/clientpositive/spark/join_reorder2.q.out @@ -214,9 +214,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 3), Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) #### 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 a607721..f1bf3c9 100644 --- ql/src/test/results/clientpositive/spark/join_reorder3.q.out +++ ql/src/test/results/clientpositive/spark/join_reorder3.q.out @@ -214,9 +214,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 3), Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) #### 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 4f9092d..8356aa3 100644 --- ql/src/test/results/clientpositive/spark/join_vc.q.out +++ ql/src/test/results/clientpositive/spark/join_vc.q.out @@ -147,7 +147,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: 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 ebb2cae..6027de0 100644 --- ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out +++ ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out @@ -493,7 +493,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -977,7 +977,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1456,7 +1456,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 c919105..a5221cd 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out @@ -177,7 +177,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -303,8 +303,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -437,7 +437,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 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 225cfe9..a8b9b4c 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 @@ -104,7 +104,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 3) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -265,7 +265,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 3) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 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 5c541b8..9b502f1 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out @@ -281,7 +281,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 3) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -1111,7 +1111,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 3) + Reducer 3 <- Map 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 2 diff --git ql/src/test/results/clientpositive/spark/multi_insert.q.out ql/src/test/results/clientpositive/spark/multi_insert.q.out index ae4899f..32bc8c6 100644 --- ql/src/test/results/clientpositive/spark/multi_insert.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert.q.out @@ -587,7 +587,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -733,7 +733,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -879,7 +879,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1025,7 +1025,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### 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 ef3c50c..c65f672 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out @@ -230,7 +230,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 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 57a0eb5..74cdbef 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out @@ -1607,8 +1607,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -1750,7 +1750,7 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### 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 c26ce5b..aac45fa 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 @@ -291,8 +291,8 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP, 3) - Reducer 3 <- Map 5 (GROUP, 3) + Reducer 2 <- Map 4 (GROUP, 2) + Reducer 3 <- Map 5 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -539,8 +539,8 @@ STAGE PLANS: Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 4 (GROUP, 3) - Reducer 3 <- Map 5 (SORT, 3) + Reducer 2 <- Map 4 (GROUP, 2) + Reducer 3 <- Map 5 (SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -806,9 +806,9 @@ STAGE PLANS: Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 5 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 5 @@ -1164,9 +1164,9 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 5 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 5 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 4c47bc3..edeb401 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 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -754,7 +754,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -904,7 +904,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1054,7 +1054,7 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2305,10 +2305,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2432,10 +2432,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2559,10 +2559,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2686,10 +2686,10 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2821,10 +2821,10 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3059,10 +3059,10 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3297,10 +3297,10 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3535,10 +3535,10 @@ STAGE PLANS: Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 3) - Reducer 6 <- Map 1 (SORT, 3) - Reducer 3 <- Reducer 5 (SORT, 3) - Reducer 4 <- Reducer 6 (SORT, 3) + Reducer 5 <- Map 1 (SORT, 2) + Reducer 6 <- Map 1 (SORT, 2) + Reducer 3 <- Reducer 5 (SORT, 2) + Reducer 4 <- Reducer 6 (SORT, 2) #### 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 94afbd9..bda569d 100644 --- ql/src/test/results/clientpositive/spark/multi_join_union.q.out +++ ql/src/test/results/clientpositive/spark/multi_join_union.q.out @@ -80,7 +80,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### 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 7968853..ce98ba0 100644 --- ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out +++ ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out @@ -228,7 +228,7 @@ STAGE PLANS: Spark Edges: Reducer 4 <- Map 3 (GROUP, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1519,7 +1519,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1766,7 +1766,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 a8f9b5b..fbbd17c 100644 --- ql/src/test/results/clientpositive/spark/outer_join_ppr.q.out +++ ql/src/test/results/clientpositive/spark/outer_join_ppr.q.out @@ -588,7 +588,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 c751aae..390aeb1 100644 --- ql/src/test/results/clientpositive/spark/parquet_join.q.out +++ ql/src/test/results/clientpositive/spark/parquet_join.q.out @@ -76,7 +76,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 3f66b46..0489117 100644 --- ql/src/test/results/clientpositive/spark/pcr.q.out +++ ql/src/test/results/clientpositive/spark/pcr.q.out @@ -2753,7 +2753,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -3061,7 +3061,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (SORT, 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 3fd3d0d..3e1e65f 100644 --- ql/src/test/results/clientpositive/spark/ppd_join.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join.q.out @@ -568,7 +568,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 287a540..56e5d24 100644 --- ql/src/test/results/clientpositive/spark/ppd_join2.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join2.q.out @@ -1740,8 +1740,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) #### 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 56787b0..afab637 100644 --- ql/src/test/results/clientpositive/spark/ppd_join3.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join3.q.out @@ -1802,8 +1802,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) #### 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 ec9fe17..4855a31 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 (SORT, 3) - Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### 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 2d7f4a0..5943e7c 100644 --- ql/src/test/results/clientpositive/spark/ppd_join5.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join5.q.out @@ -51,7 +51,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: @@ -159,7 +159,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: 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 7562352..c102915 100644 --- ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out @@ -501,8 +501,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -875,8 +875,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1249,8 +1249,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### 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 e9fb7c2..d3c3edd 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out @@ -142,7 +142,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 013da20..2733226 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out @@ -262,7 +262,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 f469c14..a0b15ea 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out @@ -256,7 +256,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 5fa0d04..1c67144 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out @@ -447,8 +447,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) #### 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 3fe2a95..6c5b976 100644 --- ql/src/test/results/clientpositive/spark/ppd_transform.q.out +++ ql/src/test/results/clientpositive/spark/ppd_transform.q.out @@ -204,7 +204,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### 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 c094108..120be56 100644 --- ql/src/test/results/clientpositive/spark/router_join_ppr.q.out +++ ql/src/test/results/clientpositive/spark/router_join_ppr.q.out @@ -596,7 +596,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -977,7 +977,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1358,7 +1358,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 b6abe84..9a90ebd 100644 --- ql/src/test/results/clientpositive/spark/sample10.q.out +++ ql/src/test/results/clientpositive/spark/sample10.q.out @@ -89,7 +89,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/sample8.q.out ql/src/test/results/clientpositive/spark/sample8.q.out index 7dd7109..b741278 100644 --- ql/src/test/results/clientpositive/spark/sample8.q.out +++ ql/src/test/results/clientpositive/spark/sample8.q.out @@ -807,7 +807,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -898,7 +898,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 9b2c1a0..f4cb111 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 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -228,8 +228,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -332,8 +332,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -428,8 +428,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -535,8 +535,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -634,8 +634,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -737,8 +737,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -837,8 +837,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -967,7 +967,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1055,8 +1055,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1155,8 +1155,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1288,8 +1288,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1451,7 +1451,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 3) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1539,8 +1539,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1668,8 +1668,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1800,8 +1800,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1932,8 +1932,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2064,8 +2064,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2198,8 +2198,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2343,9 +2343,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2503,7 +2503,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### 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 70d3623..8e735e4 100644 --- ql/src/test/results/clientpositive/spark/skewjoin.q.out +++ ql/src/test/results/clientpositive/spark/skewjoin.q.out @@ -242,7 +242,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -374,7 +374,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -498,7 +498,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -611,7 +611,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -799,7 +799,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -999,7 +999,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -1105,7 +1105,7 @@ STAGE PLANS: Stage: Stage-5 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 10 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 10 (PARTITION-LEVEL SORT, 2) #### 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 11e852e..bb5745d 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 @@ -201,8 +201,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -342,8 +342,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -493,8 +493,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out index 428f421..692afcc 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out @@ -187,8 +187,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -322,8 +322,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 3), Map 7 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1), Reducer 6 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -453,8 +453,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 3), Map 7 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1), Reducer 6 (GROUP, 1) #### 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 17485cf..527dbd0 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 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out index 4680b12..68821f9 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 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -227,8 +227,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -362,8 +362,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 3), Map 7 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1), Reducer 6 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -493,8 +493,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 3), Map 7 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1), Reducer 6 (GROUP, 1) #### 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 f83dcdd..443c825 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out @@ -249,8 +249,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out index 74c650d..60c4677 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 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out index 8d38d23..f7ce7f4 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out @@ -190,8 +190,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -320,9 +320,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 3), Map 7 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3), Reducer 6 (GROUP, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -461,9 +461,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL SORT, 3) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 3), Map 7 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3), Reducer 6 (GROUP, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 6 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out index cd92748..79f1846 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out @@ -191,8 +191,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out index 564231d..a009d46 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out @@ -185,8 +185,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 3), Map 6 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out index 16ee925..30be262 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out @@ -199,8 +199,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### 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 18a720b..3ffbb82 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out @@ -153,7 +153,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -274,7 +274,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 1) Reducer 3 <- Reducer 2 (GROUP, 1) #### 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 ca2d4c9..1a1340e 100644 --- ql/src/test/results/clientpositive/spark/temp_table_join1.q.out +++ ql/src/test/results/clientpositive/spark/temp_table_join1.q.out @@ -40,7 +40,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -136,7 +136,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -234,7 +234,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/union25.q.out ql/src/test/results/clientpositive/spark/union25.q.out index 854638b..67416a3 100644 --- ql/src/test/results/clientpositive/spark/union25.q.out +++ ql/src/test/results/clientpositive/spark/union25.q.out @@ -66,8 +66,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (GROUP, 3), Map 5 (GROUP, 3) - Reducer 2 <- Map 1 (GROUP, 3), Reducer 4 (GROUP, 3) + Reducer 4 <- Map 3 (GROUP, 2), Map 5 (GROUP, 2) + Reducer 2 <- Map 1 (GROUP, 2), Reducer 4 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/union33.q.out ql/src/test/results/clientpositive/spark/union33.q.out index 7c6ac28..fbdd18f 100644 --- ql/src/test/results/clientpositive/spark/union33.q.out +++ ql/src/test/results/clientpositive/spark/union33.q.out @@ -187,8 +187,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/union_remove_19.q.out ql/src/test/results/clientpositive/spark/union_remove_19.q.out index 9debf33..30e63a0 100644 --- ql/src/test/results/clientpositive/spark/union_remove_19.q.out +++ ql/src/test/results/clientpositive/spark/union_remove_19.q.out @@ -261,8 +261,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 4 <- Map 3 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -430,8 +430,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 4 <- Map 3 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### 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 44c03a8..94d5f28 100644 --- ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out +++ ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out @@ -119,7 +119,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out index c9ba3c8..cb2d56b 100644 --- ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out +++ ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out @@ -44,7 +44,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -149,7 +149,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out index 279c61a..6ea1555 100644 --- ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out +++ ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out @@ -119,7 +119,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out index 32bddd3..7ee669b 100644 --- ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out +++ ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out @@ -119,7 +119,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### 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 5f025cb..aba0256 100644 --- ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out @@ -231,7 +231,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 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 4bea985..3363c8b 100644 --- ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out @@ -115,7 +115,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/vectorization_13.q.out ql/src/test/results/clientpositive/spark/vectorization_13.q.out index ce10280..8426b6e 100644 --- ql/src/test/results/clientpositive/spark/vectorization_13.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_13.q.out @@ -326,7 +326,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: 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 269c4e3..5f41bc9 100644 --- ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out @@ -1866,7 +1866,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -2074,7 +2074,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -2329,7 +2329,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: @@ -2662,7 +2662,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: diff --git ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out index 529204f..8a22685 100644 --- ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out @@ -228,8 +228,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -513,9 +513,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 5 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -831,7 +831,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1087,8 +1087,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1375,8 +1375,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1672,9 +1672,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1937,8 +1937,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2250,8 +2250,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 3) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2572,8 +2572,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2849,8 +2849,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3130,8 +3130,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3418,9 +3418,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3745,8 +3745,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -4130,9 +4130,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 3), Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -4457,8 +4457,8 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (GROUP, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -4728,9 +4728,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -5197,11 +5197,11 @@ STAGE PLANS: Stage: Stage-2 Spark Edges: - Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 7 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 6 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Reducer 7 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -5711,9 +5711,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -6067,10 +6067,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -6421,9 +6421,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -6770,10 +6770,10 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) - Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -7169,9 +7169,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -7518,9 +7518,9 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 3) - Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 3) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 diff --git spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java index 9757ced..57bc2be 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java @@ -72,4 +72,9 @@ * Get the count of executors */ Future getExecutorCount(); + + /** + * Get default parallelism. For standalone mode, this can be used to get total number of cores. + */ + Future getDefaultParallelism(); } diff --git spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java index 54eacfd..aa08029 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java @@ -155,6 +155,11 @@ public void stop() { return submit(new GetExecutorCountJob()); } + @Override + public Future getDefaultParallelism() { + return submit(new GetDefaultParallelismJob()); + } + void cancel(String jobId) { protocol.cancel(jobId); } @@ -489,10 +494,20 @@ public Serializable call(JobContext jc) throws Exception { @Override public Integer call(JobContext jc) throws Exception { - int count = jc.sc().sc().getExecutorMemoryStatus().size(); + // minus 1 here otherwise driver is also counted as an executor + int count = jc.sc().sc().getExecutorMemoryStatus().size() - 1; return Integer.valueOf(count); } } + private static class GetDefaultParallelismJob implements Job { + private static final long serialVersionUID = 1L; + + @Override + public Integer call(JobContext jc) throws Exception { + return jc.sc().sc().defaultParallelism(); + } + } + }