Index: . =================================================================== --- . (revision 1654728) +++ . (working copy) Property changes on: . ___________________________________________________________________ Modified: svn:mergeinfo Merged /hive/branches/spark:r1651243-1654414 Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java =================================================================== --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 1654728) +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy) @@ -1998,12 +1998,27 @@ "hive.tez.exec.inplace.progress", true, "Updates tez job execution progress in-place in the terminal."), - SPARK_CLIENT_FUTURE_TIMEOUT( - "hive.spark.client.future.timeout", - "60s", - new TimeValidator(TimeUnit.SECONDS), - "remote spark client JobHandle future timeout value in seconds.") - ; + SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout", + "60s", new TimeValidator(TimeUnit.SECONDS), + "Timeout for requests from Hive client to remote Spark driver."), + SPARK_JOB_MONITOR_TIMEOUT("hive.spark.job.monitor.timeout", + "60s", new TimeValidator(TimeUnit.SECONDS), + "Timeout for job monitor to get Spark job state."), + SPARK_RPC_CLIENT_CONNECT_TIMEOUT("hive.spark.client.connect.timeout", + "1000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "Timeout for remote Spark driver in connecting back to Hive client."), + SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT("hive.spark.client.server.connect.timeout", + "20000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "Timeout for handshake between Hive client and remote Spark driver. Checked by both processes."), + SPARK_RPC_SECRET_RANDOM_BITS("hive.spark.client.secret.bits", "256", + "Number of bits of randomness in the generated secret for communication between Hive client and remote Spark driver. " + + "Rounded down to the nearest multiple of 8."), + SPARK_RPC_MAX_THREADS("hive.spark.client.rpc.threads", 8, + "Maximum number of threads for remote Spark driver's RPC event loop."), + SPARK_RPC_MAX_MESSAGE_SIZE("hive.spark.client.rpc.max.size", 50 * 1024 * 1024, + "Maximum message size in bytes for communication between Hive client and remote Spark driver. Default is 50MB."), + SPARK_RPC_CHANNEL_LOG_LEVEL("hive.spark.client.channel.log.level", null, + "Channel logging level for remote Spark driver. One of {DEBUG, ERROR, INFO, TRACE, WARN}."); public final String varname; private final String defaultExpr; Index: hbase-handler/pom.xml =================================================================== --- hbase-handler/pom.xml (revision 1654728) +++ hbase-handler/pom.xml (working copy) Property changes on: hbase-handler/pom.xml ___________________________________________________________________ Modified: svn:mergeinfo Merged /hive/branches/spark/hbase-handler/pom.xml:r1651243-1654414 Index: itests/src/test/resources/testconfiguration.properties =================================================================== --- itests/src/test/resources/testconfiguration.properties (revision 1654728) +++ itests/src/test/resources/testconfiguration.properties (working copy) @@ -675,6 +675,7 @@ join35.q, \ join36.q, \ join37.q, \ + join38.q, \ join39.q, \ join4.q, \ join40.q, \ @@ -699,9 +700,11 @@ join_empty.q, \ join_filters_overlap.q, \ join_hive_626.q, \ + join_literals.q, \ join_map_ppr.q, \ join_merge_multi_expressions.q, \ join_merging.q, \ + join_nullsafe.q, \ join_rc.q, \ join_reorder.q, \ join_reorder2.q, \ @@ -881,6 +884,7 @@ stats_partscan_1_23.q, \ statsfs.q, \ subquery_exists.q, \ + subquery_in.q, \ subquery_multiinsert.q, \ table_access_keys_stats.q, \ temp_table.q, \ Index: itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java =================================================================== --- itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (revision 1654728) +++ itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (working copy) @@ -78,6 +78,8 @@ import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; +import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; import org.apache.hadoop.hive.ql.lockmgr.zookeeper.CuratorFrameworkSingleton; import org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager; import org.apache.hadoop.hive.ql.metadata.Hive; @@ -149,6 +151,7 @@ private boolean miniMr = false; private String hadoopVer = null; private QTestSetup setup = null; + private SparkSession sparkSession = null; private boolean isSessionStateStarted = false; private static final String javaVersion = getJavaVersion(); @@ -418,6 +421,15 @@ } setup.tearDown(); + if (sparkSession != null) { + try { + SparkSessionManagerImpl.getInstance().closeSession(sparkSession); + } catch (Exception ex) { + LOG.error("Error closing spark session.", ex); + } finally { + sparkSession = null; + } + } if (mr != null) { mr.shutdown(); mr = null; @@ -834,7 +846,7 @@ HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER, "org.apache.hadoop.hive.ql.security.DummyAuthenticator"); Utilities.clearWorkMap(); - CliSessionState ss = new CliSessionState(conf); + CliSessionState ss = createSessionState(); assert ss != null; ss.in = System.in; @@ -864,6 +876,9 @@ SessionState oldSs = SessionState.get(); if (oldSs != null && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.spark)) { + sparkSession = oldSs.getSparkSession(); + ss.setSparkSession(sparkSession); + oldSs.setSparkSession(null); oldSs.close(); } @@ -882,6 +897,29 @@ return outf.getAbsolutePath(); } + private CliSessionState createSessionState() { + return new CliSessionState(conf) { + public void setSparkSession(SparkSession sparkSession) { + super.setSparkSession(sparkSession); + if (sparkSession != null) { + try { + // Wait a little for cluster to init, at most 4 minutes + long endTime = System.currentTimeMillis() + 240000; + while (sparkSession.getMemoryAndCores().getSecond() <= 1) { + if (System.currentTimeMillis() >= endTime) { + LOG.error("Timed out waiting for Spark cluster to init"); + break; + } + Thread.sleep(100); + } + } catch (Exception e) { + LOG.error(e); + } + } + } + }; + } + private CliSessionState startSessionState() throws IOException { @@ -890,7 +928,7 @@ String execEngine = conf.get("hive.execution.engine"); conf.set("hive.execution.engine", "mr"); - CliSessionState ss = new CliSessionState(conf); + CliSessionState ss = createSessionState(); assert ss != null; ss.in = System.in; ss.out = System.out; @@ -898,6 +936,9 @@ SessionState oldSs = SessionState.get(); if (oldSs != null && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.spark)) { + sparkSession = oldSs.getSparkSession(); + ss.setSparkSession(sparkSession); + oldSs.setSparkSession(null); oldSs.close(); } if (oldSs != null && oldSs.out != null && oldSs.out != System.out) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (working copy) @@ -31,7 +31,6 @@ import java.io.DataInput; import java.io.EOFException; import java.io.File; -import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; @@ -237,8 +236,13 @@ // prevent instantiation } - private static Map gWorkMap = Collections - .synchronizedMap(new HashMap()); + private static ThreadLocal> gWorkMap = + new ThreadLocal>() { + protected Map initialValue() { + return new HashMap(); + } + }; + private static final String CLASS_NAME = Utilities.class.getName(); private static final Log LOG = LogFactory.getLog(CLASS_NAME); @@ -345,7 +349,7 @@ */ public static void setBaseWork(Configuration conf, String name, BaseWork work) { Path path = getPlanPath(conf, name); - gWorkMap.put(path, work); + gWorkMap.get().put(path, work); } /** @@ -357,15 +361,14 @@ * @throws RuntimeException if the configuration files are not proper or if plan can not be loaded */ private static BaseWork getBaseWork(Configuration conf, String name) { - BaseWork gWork = null; Path path = null; InputStream in = null; try { path = getPlanPath(conf, name); LOG.info("PLAN PATH = " + path); assert path != null; - if (!gWorkMap.containsKey(path) - || HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) { + BaseWork gWork = gWorkMap.get().get(path); + if (gWork == null) { Path localPath; if (conf.getBoolean("mapreduce.task.uberized", false) && name.equals(REDUCE_PLAN_NAME)) { localPath = new Path(name); @@ -415,10 +418,9 @@ } else if (name.contains(MERGE_PLAN_NAME)) { gWork = deserializePlan(in, MapWork.class, conf); } - gWorkMap.put(path, gWork); - } else { + gWorkMap.get().put(path, gWork); + } else if (LOG.isDebugEnabled()) { LOG.debug("Found plan in cache for name: " + name); - gWork = gWorkMap.get(path); } return gWork; } catch (FileNotFoundException fnf) { @@ -710,7 +712,7 @@ } // Cache the plan in this process - gWorkMap.put(planPath, w); + gWorkMap.get().put(planPath, w); return planPath; } catch (Exception e) { String msg = "Error caching " + name + ": " + e; @@ -3629,15 +3631,15 @@ Path mapPath = getPlanPath(conf, MAP_PLAN_NAME); Path reducePath = getPlanPath(conf, REDUCE_PLAN_NAME); if (mapPath != null) { - gWorkMap.remove(mapPath); + gWorkMap.get().remove(mapPath); } if (reducePath != null) { - gWorkMap.remove(reducePath); + gWorkMap.get().remove(reducePath); } } public static void clearWorkMap() { - gWorkMap.clear(); + gWorkMap.get().clear(); } /** Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java (working copy) @@ -21,14 +21,18 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.TimeUnit; import org.apache.commons.compress.utils.CharsetNames; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.spark.client.rpc.RpcConfiguration; import org.apache.spark.SparkConf; import org.apache.spark.SparkException; @@ -95,7 +99,8 @@ } } - // load properties from hive configurations. + // load properties from hive configurations, including both spark.* properties + // and properties for remote driver RPC. for (Map.Entry entry : hiveConf) { String propertyName = entry.getKey(); if (propertyName.startsWith("spark")) { @@ -105,6 +110,13 @@ "load spark configuration from hive configuration (%s -> %s).", propertyName, value)); } + if (RpcConfiguration.HIVE_SPARK_RSC_CONFIGS.contains(propertyName)) { + String value = RpcConfiguration.getValue(hiveConf, propertyName); + sparkConf.put(propertyName, value); + LOG.info(String.format( + "load RPC configuration from hive configuration (%s -> %s).", + propertyName, value)); + } } return sparkConf; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java (working copy) @@ -32,6 +32,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.impl.JobMetricsListener; +import org.apache.hadoop.hive.ql.exec.spark.status.impl.LocalSparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.impl.LocalSparkJobStatus; import org.apache.hadoop.hive.ql.io.HiveKey; import org.apache.hadoop.hive.ql.plan.BaseWork; @@ -136,7 +137,7 @@ int jobId = future.jobIds().get(0); LocalSparkJobStatus sparkJobStatus = new LocalSparkJobStatus( sc, jobId, jobMetricsListener, sparkCounters, plan.getCachedRDDIds(), future); - return new SparkJobRef(Integer.toString(jobId), sparkJobStatus); + return new LocalSparkJobRef(Integer.toString(jobId), hiveConf, sparkJobStatus, sc); } /** Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java (working copy) @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; +import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobStatus; import org.apache.hadoop.hive.ql.io.HiveKey; import org.apache.hadoop.hive.ql.plan.BaseWork; @@ -76,10 +77,13 @@ private transient List localJars = new ArrayList(); private transient List localFiles = new ArrayList(); + private final transient long sparkClientTimtout; + RemoteHiveSparkClient(HiveConf hiveConf, Map conf) throws IOException, SparkException { this.hiveConf = hiveConf; sparkConf = HiveSparkClientFactory.generateSparkConf(conf); - remoteClient = SparkClientFactory.createClient(conf); + remoteClient = SparkClientFactory.createClient(conf, hiveConf); + sparkClientTimtout = hiveConf.getTimeVar(HiveConf.ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT, TimeUnit.SECONDS); } @Override @@ -89,17 +93,14 @@ @Override public int getExecutorCount() throws Exception { - long timeout = hiveConf.getTimeVar(HiveConf.ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT, TimeUnit.SECONDS); Future handler = remoteClient.getExecutorCount(); - return handler.get(timeout, TimeUnit.SECONDS).intValue(); + return handler.get(sparkClientTimtout, TimeUnit.SECONDS).intValue(); } @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); + return handler.get(sparkClientTimtout, TimeUnit.SECONDS); } @Override @@ -118,11 +119,10 @@ byte[] scratchDirBytes = KryoSerializer.serialize(emptyScratchDir); byte[] sparkWorkBytes = KryoSerializer.serialize(sparkWork); - long timeout = hiveConf.getTimeVar(HiveConf.ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT, TimeUnit.SECONDS); - - JobHandle jobHandle = remoteClient.submit( - new JobStatusJob(jobConfBytes, scratchDirBytes, sparkWorkBytes)); - return new SparkJobRef(jobHandle.getClientJobId(), new RemoteSparkJobStatus(remoteClient, jobHandle, timeout)); + JobStatusJob job = new JobStatusJob(jobConfBytes, scratchDirBytes, sparkWorkBytes); + JobHandle jobHandle = remoteClient.submit(job); + RemoteSparkJobStatus sparkJobStatus = new RemoteSparkJobStatus(remoteClient, jobHandle, sparkClientTimtout); + return new RemoteSparkJobRef(hiveConf, jobHandle, sparkJobStatus); } private void refreshLocalResources(SparkWork sparkWork, HiveConf conf) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java (working copy) @@ -49,7 +49,7 @@ import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManager; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor; +import org.apache.hadoop.hive.ql.exec.spark.status.LocalSparkJobMonitor; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; import org.apache.hadoop.hive.ql.log.PerfLogger; @@ -65,7 +65,6 @@ import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.StatsWork; -import org.apache.hadoop.hive.ql.plan.UnionWork; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.stats.StatsFactory; import org.apache.hadoop.util.StringUtils; @@ -102,19 +101,20 @@ SparkJobRef jobRef = sparkSession.submit(driverContext, sparkWork); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); + rc = jobRef.monitorJob(); SparkJobStatus sparkJobStatus = jobRef.getSparkJobStatus(); - if (sparkJobStatus != null) { - SparkJobMonitor monitor = new SparkJobMonitor(sparkJobStatus); - rc = monitor.startMonitor(); + if (rc == 0) { + sparkCounters = sparkJobStatus.getCounter(); // for RSC, we should get the counters after job has finished - sparkCounters = sparkJobStatus.getCounter(); SparkStatistics sparkStatistics = sparkJobStatus.getSparkStatistics(); if (LOG.isInfoEnabled() && sparkStatistics != null) { LOG.info(String.format("=====Spark Job[%s] statistics=====", jobRef.getJobId())); logSparkStatistic(sparkStatistics); } - sparkJobStatus.cleanup(); + } else if (rc == 2) { // Cancel job if the monitor found job submission timeout. + jobRef.cancelJob(); } + sparkJobStatus.cleanup(); } catch (Exception e) { String msg = "Failed to execute spark task, with exception '" + Utilities.getNameMessage(e) + "'"; @@ -124,6 +124,7 @@ LOG.error(msg, e); rc = 1; } finally { + Utilities.clearWork(conf); if (sparkSession != null && sparkSessionManager != null) { rc = close(rc); try { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java (working copy) @@ -17,14 +17,13 @@ */ package org.apache.hadoop.hive.ql.exec.spark.session; +import org.apache.hadoop.hive.common.ObjectPair; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.SparkWork; -import scala.Tuple2; - public interface SparkSession { /** * Initializes a Spark session for DAG execution. @@ -44,10 +43,10 @@ * Get Spark shuffle memory per task, and total number of cores. This * information can be used to estimate how many reducers a task can have. * - * @return a tuple, the first element is the shuffle memory per task in bytes, + * @return an object pair, the first element is the shuffle memory per task in bytes, * the second element is the number of total cores usable by the client */ - Tuple2 getMemoryAndCores() throws Exception; + ObjectPair getMemoryAndCores() throws Exception; /** * @return true if the session is open and ready to submit jobs. Index: 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 (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java (working copy) @@ -22,6 +22,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.common.ObjectPair; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClient; @@ -32,8 +33,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.util.Utils; -import scala.Tuple2; - import com.google.common.base.Preconditions; public class SparkSessionImpl implements SparkSession { @@ -66,12 +65,12 @@ } @Override - public Tuple2 getMemoryAndCores() throws Exception { + public ObjectPair getMemoryAndCores() throws Exception { SparkConf sparkConf = hiveSparkClient.getSparkConf(); int numExecutors = hiveSparkClient.getExecutorCount(); // at start-up, we may be unable to get number of executors if (numExecutors <= 0) { - return new Tuple2(-1L, -1); + return new ObjectPair(-1L, -1); } int executorMemoryInMB = Utils.memoryStringToMb( sparkConf.get("spark.executor.memory", "512m")); @@ -94,7 +93,7 @@ LOG.info("Spark cluster current has executors: " + numExecutors + ", total cores: " + totalCores + ", memory per executor: " + executorMemoryInMB + "M, memoryFraction: " + memoryFraction); - return new Tuple2(Long.valueOf(memoryPerTaskInBytes), + return new ObjectPair(Long.valueOf(memoryPerTaskInBytes), Integer.valueOf(totalCores)); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java (working copy) @@ -97,28 +97,16 @@ * - create a new session and add it to the list. */ @Override - public SparkSession getSession(SparkSession existingSession, HiveConf conf, - boolean doOpen) throws HiveException { + public SparkSession getSession(SparkSession existingSession, HiveConf conf, boolean doOpen) + throws HiveException { setup(conf); if (existingSession != null) { - if (canReuseSession(existingSession, conf)) { - // Open the session if it is closed. - if (!existingSession.isOpen() && doOpen) { - existingSession.open(conf); - } - - Preconditions.checkState(createdSessions.contains(existingSession)); - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Existing session (%s) is reused.", - existingSession.getSessionId())); - } - return existingSession; - } else { - // Close the session, as the client is holding onto a session that can't be used - // by the client anymore. - closeSession(existingSession); + // Open the session if it is closed. + if (!existingSession.isOpen() && doOpen) { + existingSession.open(conf); } + return existingSession; } SparkSession sparkSession = new SparkSessionImpl(); @@ -134,24 +122,6 @@ return sparkSession; } - /** - * Currently we only match the userNames in existingSession conf and given conf. - */ - private boolean canReuseSession(SparkSession existingSession, HiveConf conf) throws HiveException { - try { - UserGroupInformation newUgi = Utils.getUGI(); - String newUserName = newUgi.getShortUserName(); - - // TODO this we need to store the session username somewhere else as getUGIForConf never used the conf - UserGroupInformation ugiInSession = Utils.getUGI(); - String userNameInSession = ugiInSession.getShortUserName(); - - return newUserName.equals(userNameInSession); - } catch (Exception ex) { - throw new HiveException("Failed to get user info from HiveConf.", ex); - } - } - @Override public void returnSession(SparkSession sparkSession) throws HiveException { // In this particular SparkSessionManager implementation, we don't recycle Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java (working copy) @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.exec.spark.status; + +import java.util.Map; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.spark.JobExecutionStatus; + +/** + * LocalSparkJobMonitor monitor a single Spark job status in a loop until job finished/failed/killed. + * It print current job status to console and sleep current thread between monitor interval. + */ +public class LocalSparkJobMonitor extends SparkJobMonitor { + + private SparkJobStatus sparkJobStatus; + + public LocalSparkJobMonitor(HiveConf hiveConf, SparkJobStatus sparkJobStatus) { + super(hiveConf); + this.sparkJobStatus = sparkJobStatus; + } + + public int startMonitor() { + boolean running = false; + boolean done = false; + int rc = 0; + JobExecutionStatus lastState = null; + Map lastProgressMap = null; + + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); + + long startTime = System.currentTimeMillis(); + + while (true) { + try { + JobExecutionStatus state = sparkJobStatus.getState(); + if (LOG.isDebugEnabled()) { + console.printInfo("state = " + state); + } + + if (state == null) { + long timeCount = (System.currentTimeMillis() - startTime)/1000; + if (timeCount > monitorTimeoutInteval) { + LOG.info("Job hasn't been submitted after " + timeCount + "s. Aborting it."); + console.printError("Status: " + state); + running = false; + done = true; + rc = 2; + break; + } + } else if (state != lastState || state == JobExecutionStatus.RUNNING) { + lastState = state; + Map progressMap = sparkJobStatus.getSparkStageProgress(); + + switch (state) { + case RUNNING: + if (!running) { + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); + // print job stages. + console.printInfo("\nQuery Hive on Spark job[" + + sparkJobStatus.getJobId() + "] stages:"); + for (int stageId : sparkJobStatus.getStageIds()) { + console.printInfo(Integer.toString(stageId)); + } + + console.printInfo("\nStatus: Running (Hive on Spark job[" + + sparkJobStatus.getJobId() + "])"); + running = true; + + console.printInfo("Job Progress Format\nCurrentTime StageId_StageAttemptId: " + + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]"); + } + + printStatus(progressMap, lastProgressMap); + lastProgressMap = progressMap; + break; + case SUCCEEDED: + printStatus(progressMap, lastProgressMap); + lastProgressMap = progressMap; + double duration = (System.currentTimeMillis() - startTime) / 1000.0; + console.printInfo("Status: Finished successfully in " + + String.format("%.2f seconds", duration)); + running = false; + done = true; + break; + case FAILED: + console.printError("Status: Failed"); + running = false; + done = true; + rc = 3; + break; + case UNKNOWN: + console.printError("Status: Unknown"); + running = false; + done = true; + rc = 4; + break; + } + } + if (!done) { + Thread.sleep(checkInterval); + } + } catch (Exception e) { + String msg = " with exception '" + Utilities.getNameMessage(e) + "'"; + msg = "Failed to monitor Job[ " + sparkJobStatus.getJobId() + "]" + msg; + + // Has to use full name to make sure it does not conflict with + // org.apache.commons.lang.StringUtils + LOG.error(msg, e); + console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); + rc = 1; + done = true; + } finally { + if (done) { + break; + } + } + } + + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); + return rc; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java (working copy) @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec.spark.status; + +import java.util.Map; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobStatus; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hive.spark.client.JobHandle; +import org.apache.spark.JobExecutionStatus; + +/** + * RemoteSparkJobMonitor monitor a RSC remote job status in a loop until job finished/failed/killed. + * It print current job status to console and sleep current thread between monitor interval. + */ +public class RemoteSparkJobMonitor extends SparkJobMonitor { + + private RemoteSparkJobStatus sparkJobStatus; + + public RemoteSparkJobMonitor(HiveConf hiveConf, RemoteSparkJobStatus sparkJobStatus) { + super(hiveConf); + this.sparkJobStatus = sparkJobStatus; + } + + @Override + public int startMonitor() { + boolean running = false; + boolean done = false; + int rc = 0; + Map lastProgressMap = null; + + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); + + long startTime = System.currentTimeMillis(); + + while (true) { + try { + JobHandle.State state = sparkJobStatus.getRemoteJobState(); + if (LOG.isDebugEnabled()) { + console.printInfo("state = " + state); + } + + switch (state) { + case SENT: + case QUEUED: + long timeCount = (System.currentTimeMillis() - startTime) / 1000; + if ((timeCount > monitorTimeoutInteval)) { + LOG.info("Job hasn't been submitted after " + timeCount + "s. Aborting it."); + console.printError("Status: " + state); + running = false; + done = true; + rc = 2; + } + break; + case STARTED: + JobExecutionStatus sparkJobState = sparkJobStatus.getState(); + if (sparkJobState == JobExecutionStatus.RUNNING) { + Map progressMap = sparkJobStatus.getSparkStageProgress(); + if (!running) { + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); + // print job stages. + console.printInfo("\nQuery Hive on Spark job[" + + sparkJobStatus.getJobId() + "] stages:"); + for (int stageId : sparkJobStatus.getStageIds()) { + console.printInfo(Integer.toString(stageId)); + } + + console.printInfo("\nStatus: Running (Hive on Spark job[" + + sparkJobStatus.getJobId() + "])"); + running = true; + + console.printInfo("Job Progress Format\nCurrentTime StageId_StageAttemptId: " + + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]"); + } + + printStatus(progressMap, lastProgressMap); + lastProgressMap = progressMap; + } + break; + case SUCCEEDED: + Map progressMap = sparkJobStatus.getSparkStageProgress(); + printStatus(progressMap, lastProgressMap); + lastProgressMap = progressMap; + double duration = (System.currentTimeMillis() - startTime) / 1000.0; + console.printInfo("Status: Finished successfully in " + + String.format("%.2f seconds", duration)); + running = false; + done = true; + break; + case FAILED: + console.printError("Status: Failed"); + running = false; + done = true; + rc = 3; + break; + } + + if (!done) { + Thread.sleep(checkInterval); + } + } catch (Exception e) { + String msg = " with exception '" + Utilities.getNameMessage(e) + "'"; + msg = "Failed to monitor Job[ " + sparkJobStatus.getJobId() + "]" + msg; + + // Has to use full name to make sure it does not conflict with + // org.apache.commons.lang.StringUtils + LOG.error(msg, e); + console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); + rc = 1; + done = true; + } finally { + if (done) { + break; + } + } + } + + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); + return rc; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java (working copy) @@ -15,8 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.hadoop.hive.ql.exec.spark.status; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.spark.status.impl.LocalSparkJobStatus; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.session.SessionState; + import java.text.SimpleDateFormat; import java.util.Date; import java.util.HashSet; @@ -24,142 +32,30 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; -import org.apache.spark.JobExecutionStatus; +abstract class SparkJobMonitor { -/** - * SparkJobMonitor monitor a single Spark job status in a loop until job finished/failed/killed. - * It print current job status to console and sleep current thread between monitor interval. - */ -public class SparkJobMonitor { + protected static final String CLASS_NAME = SparkJobMonitor.class.getName(); + protected static final Log LOG = LogFactory.getLog(CLASS_NAME); + protected static SessionState.LogHelper console = new SessionState.LogHelper(LOG); + protected final PerfLogger perfLogger = PerfLogger.getPerfLogger(); + protected final int checkInterval = 1000; + protected final long monitorTimeoutInteval; - private static final String CLASS_NAME = SparkJobMonitor.class.getName(); - private static final Log LOG = LogFactory.getLog(CLASS_NAME); - - private transient LogHelper console; - private final PerfLogger perfLogger = PerfLogger.getPerfLogger(); - private final int checkInterval = 1000; + private Set completed = new HashSet(); private final int printInterval = 3000; private long lastPrintTime; - private Set completed; - private SparkJobStatus sparkJobStatus; - - public SparkJobMonitor(SparkJobStatus sparkJobStatus) { - this.sparkJobStatus = sparkJobStatus; - console = new LogHelper(LOG); + protected SparkJobMonitor(HiveConf hiveConf) { + monitorTimeoutInteval = hiveConf.getTimeVar(HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT, TimeUnit.SECONDS); } - public int startMonitor() { - completed = new HashSet(); + public abstract int startMonitor(); - boolean running = false; - boolean done = false; - int rc = 0; - JobExecutionStatus lastState = null; - Map lastProgressMap = null; - long startTime = -1; + protected void printStatus(Map progressMap, + Map lastProgressMap) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); - - while (true) { - JobExecutionStatus state = sparkJobStatus.getState(); - try { - if (LOG.isDebugEnabled()) { - console.printInfo("state = " + state); - } - if (state != null && state != JobExecutionStatus.UNKNOWN - && (state != lastState || state == JobExecutionStatus.RUNNING)) { - lastState = state; - Map progressMap = sparkJobStatus.getSparkStageProgress(); - - switch (state) { - case RUNNING: - if (!running) { - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); - // print job stages. - console.printInfo("\nQuery Hive on Spark job[" - + sparkJobStatus.getJobId() + "] stages:"); - for (int stageId : sparkJobStatus.getStageIds()) { - console.printInfo(Integer.toString(stageId)); - } - - console.printInfo("\nStatus: Running (Hive on Spark job[" - + sparkJobStatus.getJobId() + "])"); - startTime = System.currentTimeMillis(); - running = true; - - console.printInfo("Job Progress Format\nCurrentTime StageId_StageAttemptId: " - + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]"); - } - - - printStatus(progressMap, lastProgressMap); - lastProgressMap = progressMap; - break; - case SUCCEEDED: - printStatus(progressMap, lastProgressMap); - lastProgressMap = progressMap; - if (startTime < 0) { - console.printInfo("Status: Finished successfully within a check interval."); - } else { - double duration = (System.currentTimeMillis() - startTime) / 1000.0; - console.printInfo("Status: Finished successfully in " - + String.format("%.2f seconds", duration)); - } - running = false; - done = true; - break; - case FAILED: - console.printError("Status: Failed"); - running = false; - done = true; - rc = 2; - break; - case UNKNOWN: - console.printError("Status: Unknown"); - running = false; - done = true; - rc = 2; - break; - } - } - if (!done) { - Thread.sleep(checkInterval); - } - } catch (Exception e) { - String msg = " with exception '" + Utilities.getNameMessage(e) + "'"; - if (state == null || state.equals(JobExecutionStatus.UNKNOWN)) { - msg = "Job Submission failed" + msg; - } else { - msg = "Ended Job = " + sparkJobStatus.getJobId() + msg; - } - - // Has to use full name to make sure it does not conflict with - // org.apache.commons.lang.StringUtils - LOG.error(msg, e); - console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - rc = 1; - } finally { - if (done) { - break; - } - } - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); - return rc; - } - - private void printStatus(Map progressMap, - Map lastProgressMap) { - // do not print duplicate status while still in middle of print interval. boolean isDuplicateState = isSameAsPreviousProgress(progressMap, lastProgressMap); boolean isPassedInterval = System.currentTimeMillis() <= lastPrintTime + printInterval; @@ -209,13 +105,13 @@ if (failed > 0) { /* tasks finished but some failed */ reportBuffer.append( - String.format( - "%s: %d(-%d)/%d Finished with failed tasks\t", - stageName, complete, failed, total)); + String.format( + "%s: %d(-%d)/%d Finished with failed tasks\t", + stageName, complete, failed, total)); } else { if (complete == total) { reportBuffer.append( - String.format("%s: %d/%d Finished\t", stageName, complete, total)); + String.format("%s: %d/%d Finished\t", stageName, complete, total)); } else { reportBuffer.append(String.format("%s: %d/%d\t", stageName, complete, total)); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java (working copy) @@ -17,36 +17,13 @@ */ package org.apache.hadoop.hive.ql.exec.spark.status; -public class SparkJobRef { +public interface SparkJobRef { - private String jobId; + public String getJobId(); - private SparkJobStatus sparkJobStatus; + public SparkJobStatus getSparkJobStatus(); - public SparkJobRef() { } + public boolean cancelJob(); - public SparkJobRef(String jobId) { - this.jobId = jobId; - } - - public SparkJobRef(String jobId, SparkJobStatus sparkJobStatus) { - this.jobId = jobId; - this.sparkJobStatus = sparkJobStatus; - } - - public String getJobId() { - return jobId; - } - - public void setJobId(String jobId) { - this.jobId = jobId; - } - - public SparkJobStatus getSparkJobStatus() { - return sparkJobStatus; - } - - public void setSparkJobStatus(SparkJobStatus sparkJobStatus) { - this.sparkJobStatus = sparkJobStatus; - } + public int monitorJob(); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java (working copy) @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec.spark.status; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistics; +import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hive.spark.counter.SparkCounters; import org.apache.spark.JobExecutionStatus; @@ -30,11 +31,11 @@ int getJobId(); - JobExecutionStatus getState(); + JobExecutionStatus getState() throws HiveException; - int[] getStageIds(); + int[] getStageIds() throws HiveException; - Map getSparkStageProgress(); + Map getSparkStageProgress() throws HiveException; SparkCounters getCounter(); Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobRef.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobRef.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobRef.java (working copy) @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.exec.spark.status.impl; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.spark.status.LocalSparkJobMonitor; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; +import org.apache.spark.api.java.JavaSparkContext; + +public class LocalSparkJobRef implements SparkJobRef { + + private final String jobId; + private final HiveConf hiveConf; + private final LocalSparkJobStatus sparkJobStatus; + private final JavaSparkContext javaSparkContext; + + public LocalSparkJobRef( + String jobId, + HiveConf hiveConf, + LocalSparkJobStatus sparkJobStatus, + JavaSparkContext javaSparkContext) { + + this.jobId = jobId; + this.hiveConf = hiveConf; + this.sparkJobStatus = sparkJobStatus; + this.javaSparkContext = javaSparkContext; + } + + @Override + public String getJobId() { + return jobId; + } + + @Override + public SparkJobStatus getSparkJobStatus() { + return sparkJobStatus; + } + + @Override + public boolean cancelJob() { + int id = Integer.parseInt(jobId); + javaSparkContext.sc().cancelJob(id); + return true; + } + + @Override + public int monitorJob() { + LocalSparkJobMonitor localSparkJobMonitor = new LocalSparkJobMonitor(hiveConf, sparkJobStatus); + return localSparkJobMonitor.startMonitor(); + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobRef.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobRef.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobRef.java (working copy) @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.exec.spark.status.impl; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.spark.status.RemoteSparkJobMonitor; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; +import org.apache.hive.spark.client.JobHandle; + +import java.io.Serializable; + +public class RemoteSparkJobRef implements SparkJobRef { + + private final String jobId; + private final HiveConf hiveConf; + private final RemoteSparkJobStatus sparkJobStatus; + private final JobHandle jobHandler; + + public RemoteSparkJobRef(HiveConf hiveConf, JobHandle jobHandler, RemoteSparkJobStatus sparkJobStatus) { + this.jobHandler = jobHandler; + this.jobId = jobHandler.getClientJobId(); + this.hiveConf = hiveConf; + this.sparkJobStatus = sparkJobStatus; + } + + @Override + public String getJobId() { + return jobId; + } + + @Override + public SparkJobStatus getSparkJobStatus() { + return sparkJobStatus; + } + + @Override + public boolean cancelJob() { + return jobHandler.cancel(true); + } + + @Override + public int monitorJob() { + RemoteSparkJobMonitor remoteSparkJobMonitor = new RemoteSparkJobMonitor(hiveConf, sparkJobStatus); + return remoteSparkJobMonitor.startMonitor(); + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java (working copy) @@ -22,6 +22,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistics; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsBuilder; +import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hive.spark.client.MetricsCollection; import org.apache.hive.spark.client.metrics.Metrics; import org.apache.hive.spark.client.metrics.ShuffleReadMetrics; @@ -42,6 +43,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; /** @@ -49,11 +51,6 @@ */ public class RemoteSparkJobStatus implements SparkJobStatus { private static final Log LOG = LogFactory.getLog(RemoteSparkJobStatus.class.getName()); - // time (in milliseconds) to wait for a spark job to be submitted on remote cluster - // after this period, we decide the job submission has failed so that client won't hang forever - private static final long WAIT_SUBMISSION_TIMEOUT = 30000; - // remember when the monitor starts - private final long startTime; private final SparkClient sparkClient; private final JobHandle jobHandle; private final transient long sparkClientTimeoutInSeconds; @@ -62,7 +59,6 @@ this.sparkClient = sparkClient; this.jobHandle = jobHandle; this.sparkClientTimeoutInSeconds = timeoutInSeconds; - startTime = System.nanoTime(); } @Override @@ -71,19 +67,19 @@ } @Override - public JobExecutionStatus getState() { + public JobExecutionStatus getState() throws HiveException { SparkJobInfo sparkJobInfo = getSparkJobInfo(); return sparkJobInfo != null ? sparkJobInfo.status() : null; } @Override - public int[] getStageIds() { + public int[] getStageIds() throws HiveException { SparkJobInfo sparkJobInfo = getSparkJobInfo(); return sparkJobInfo != null ? sparkJobInfo.stageIds() : new int[0]; } @Override - public Map getSparkStageProgress() { + public Map getSparkStageProgress() throws HiveException { Map stageProgresses = new HashMap(); for (int stageId : getStageIds()) { SparkStageInfo sparkStageInfo = getSparkStageInfo(stageId); @@ -131,32 +127,24 @@ } - private SparkJobInfo getSparkJobInfo() { + private SparkJobInfo getSparkJobInfo() throws HiveException { Integer sparkJobId = jobHandle.getSparkJobIds().size() == 1 ? jobHandle.getSparkJobIds().get(0) : null; if (sparkJobId == null) { - long duration = TimeUnit.MILLISECONDS.convert( - System.nanoTime() - startTime, TimeUnit.NANOSECONDS); - if (duration <= WAIT_SUBMISSION_TIMEOUT) { - return null; - } else { - LOG.info("Job hasn't been submitted after " + duration / 1000 + "s. Aborting it."); - jobHandle.cancel(false); - return getDefaultJobInfo(sparkJobId, JobExecutionStatus.FAILED); - } + return null; } - JobHandle getJobInfo = sparkClient.submit( + Future getJobInfo = sparkClient.run( new GetJobInfoJob(jobHandle.getClientJobId(), sparkJobId)); try { return getJobInfo.get(sparkClientTimeoutInSeconds, TimeUnit.SECONDS); - } catch (Throwable t) { - LOG.warn("Error getting job info", t); - return null; + } catch (Exception e) { + LOG.warn("Failed to get job info.", e); + throw new HiveException(e); } } private SparkStageInfo getSparkStageInfo(int stageId) { - JobHandle getStageInfo = sparkClient.submit(new GetStageInfoJob(stageId)); + Future getStageInfo = sparkClient.run(new GetStageInfoJob(stageId)); try { return getStageInfo.get(sparkClientTimeoutInSeconds, TimeUnit.SECONDS); } catch (Throwable t) { @@ -165,6 +153,10 @@ } } + public JobHandle.State getRemoteJobState() { + return jobHandle.getState(); + } + private static class GetJobInfoJob implements Job { private final String clientJobId; private final int sparkJobId; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java (revision 1654728) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java (working copy) @@ -22,6 +22,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.common.ObjectPair; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; @@ -42,8 +43,6 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; -import scala.Tuple2; - /** * SetSparkReducerParallelism determines how many reducers should * be run for a given reduce sink, clone from SetReducerParallelism. @@ -53,7 +52,7 @@ private static final Log LOG = LogFactory.getLog(SetSparkReducerParallelism.class.getName()); // Spark memory per task, and total number of cores - private Tuple2 sparkMemoryAndCores; + private ObjectPair sparkMemoryAndCores; @Override public Object process(Node nd, Stack stack, @@ -135,15 +134,15 @@ maxReducers, false); if (sparkMemoryAndCores != null && - sparkMemoryAndCores._1() > 0 && sparkMemoryAndCores._2() > 0) { + sparkMemoryAndCores.getFirst() > 0 && sparkMemoryAndCores.getSecond() > 0) { // warn the user if bytes per reducer is much larger than memory per task - if ((double) sparkMemoryAndCores._1() / bytesPerReducer < 0.5) { + if ((double) sparkMemoryAndCores.getFirst() / 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 - numReducers = Math.max(numReducers, sparkMemoryAndCores._2()); + numReducers = Math.max(numReducers, sparkMemoryAndCores.getSecond()); } numReducers = Math.min(numReducers, maxReducers); LOG.info("Set parallelism for reduce sink " + sink + " to: " + numReducers + Index: ql/src/test/results/clientpositive/spark/auto_join0.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join0.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join0.q.out (working copy) @@ -53,7 +53,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join15.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join15.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join15.q.out (working copy) @@ -42,7 +42,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join18.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join18.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join18.q.out (working copy) @@ -32,9 +32,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out (working copy) @@ -34,9 +34,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join20.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join20.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join20.q.out (working copy) @@ -66,7 +66,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 1) + Reducer 4 <- Map 3 (SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join21.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join21.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join21.q.out (working copy) @@ -57,7 +57,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 1) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/auto_join23.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join23.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join23.q.out (working copy) @@ -37,7 +37,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/auto_join26.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join26.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join26.q.out (working copy) @@ -47,7 +47,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP, 1) + Reducer 3 <- Map 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 2 Index: ql/src/test/results/clientpositive/spark/auto_join27.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join27.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join27.q.out (working copy) @@ -30,8 +30,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join28.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join28.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join28.q.out (working copy) @@ -53,7 +53,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 1) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -166,7 +166,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -279,7 +279,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 1) + Reducer 3 <- Map 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 2 @@ -389,7 +389,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 1) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/auto_join29.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join29.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join29.q.out (working copy) @@ -57,7 +57,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 1) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/auto_join30.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join30.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join30.q.out (working copy) @@ -23,7 +23,7 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 1) + Reducer 5 <- Map 4 (SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -59,7 +59,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join31.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join31.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join31.q.out (working copy) @@ -29,8 +29,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) - Reducer 7 <- Map 6 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) + Reducer 7 <- Map 6 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -90,7 +90,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 1) + Reducer 4 <- Map 3 (SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join32.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join32.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join32.q.out (working copy) @@ -54,7 +54,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/auto_join6.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join6.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join6.q.out (working copy) @@ -45,7 +45,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/auto_join7.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join7.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join7.q.out (working copy) @@ -55,7 +55,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out (working copy) @@ -12,7 +12,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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: Index: ql/src/test/results/clientpositive/spark/count.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/count.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/count.q.out (working copy) @@ -43,7 +43,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/cross_join.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cross_join.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/cross_join.q.out (working copy) @@ -138,7 +138,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ctas.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ctas.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ctas.q.out (working copy) @@ -32,7 +32,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out (working copy) @@ -14,7 +14,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -65,7 +65,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/escape_distributeby1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/escape_distributeby1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/escape_distributeby1.q.out (working copy) @@ -14,7 +14,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -65,7 +65,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/escape_sortby1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/escape_sortby1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/escape_sortby1.q.out (working copy) @@ -14,7 +14,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -64,7 +64,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby1.q.out (working copy) @@ -25,8 +25,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby10.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby10.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby10.q.out (working copy) @@ -55,10 +55,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 1) - Reducer 6 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 5 (GROUP, 1) - Reducer 4 <- Reducer 6 (GROUP, 1) + 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 Index: ql/src/test/results/clientpositive/spark/groupby11.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby11.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby11.q.out (working copy) @@ -43,10 +43,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 1) - Reducer 6 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 5 (GROUP, 1) - Reducer 4 <- Reducer 6 (GROUP, 1) + 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 Index: ql/src/test/results/clientpositive/spark/groupby2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby2.q.out (working copy) @@ -23,7 +23,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby3.q.out (working copy) @@ -41,7 +41,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -148,4 +148,4 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@dest1 #### A masked pattern was here #### -130091.0 260.182 256.10355987055016 98.0 0.0 142.92680950752379 143.06995106518903 20428.07287599999 20469.010897795582 +130091.0 260.182 256.10355987055016 98.0 0.0 142.9268095075238 143.06995106518906 20428.072876000002 20469.010897795593 Index: ql/src/test/results/clientpositive/spark/groupby4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby4.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby4.q.out (working copy) @@ -27,8 +27,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby8.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby8.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby8.q.out (working copy) @@ -39,10 +39,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 1) - Reducer 6 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 5 (GROUP, 1) - Reducer 4 <- Reducer 6 (GROUP, 1) + 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 Index: ql/src/test/results/clientpositive/spark/groupby9.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby9.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby9.q.out (working copy) @@ -39,10 +39,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 1) - Reducer 6 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 5 (GROUP, 1) - Reducer 4 <- Reducer 6 (GROUP, 1) + 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 Index: ql/src/test/results/clientpositive/spark/groupby_complex_types.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_complex_types.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_complex_types.q.out (working copy) @@ -51,9 +51,9 @@ Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 5 (GROUP, 1) - Reducer 3 <- Map 6 (GROUP, 1) - Reducer 4 <- Map 7 (GROUP, 1) + Reducer 2 <- Map 5 (GROUP, 2) + Reducer 3 <- Map 6 (GROUP, 2) + Reducer 4 <- Map 7 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 5 Index: ql/src/test/results/clientpositive/spark/groupby_complex_types_multi_single_reducer.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_complex_types_multi_single_reducer.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_complex_types_multi_single_reducer.q.out (working copy) @@ -39,8 +39,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 6 (GROUP, 1) - Reducer 4 <- Map 7 (GROUP, 1) + Reducer 2 <- Map 6 (GROUP, 2) + Reducer 4 <- Map 7 (GROUP, 2) Reducer 3 <- Reducer 2 (GROUP, 1) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### @@ -204,16 +204,16 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@dest1 #### A masked pattern was here #### -["229"] 2 -["242"] 2 -["30"] 1 -["302"] 1 -["315"] 1 -["333"] 2 -["378"] 1 -["41"] 1 +["166"] 1 +["169"] 4 +["238"] 2 +["258"] 1 +["306"] 1 +["384"] 3 +["392"] 1 +["435"] 1 +["455"] 1 ["468"] 4 -["8"] 1 PREHOOK: query: SELECT DEST2.* FROM DEST2 PREHOOK: type: QUERY PREHOOK: Input: default@dest2 Index: ql/src/test/results/clientpositive/spark/groupby_cube1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_cube1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_cube1.q.out (working copy) @@ -32,7 +32,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: 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 (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out (working copy) @@ -39,10 +39,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 1) - Reducer 6 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 5 (GROUP, 1) - Reducer 4 <- Reducer 6 (GROUP, 1) + 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 Index: ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out (working copy) @@ -67,7 +67,7 @@ Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out (working copy) @@ -39,7 +39,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out (working copy) @@ -49,7 +49,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby_position.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_position.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_position.q.out (working copy) @@ -39,8 +39,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/groupby_ppr.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_ppr.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_ppr.q.out (working copy) @@ -104,7 +104,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out (working copy) @@ -32,7 +32,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/having.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/having.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/having.q.out (working copy) @@ -12,7 +12,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/identity_project_remove_skip.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/identity_project_remove_skip.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/identity_project_remove_skip.q.out (working copy) @@ -25,7 +25,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -56,7 +56,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 1) + Reducer 4 <- Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/index_auto_self_join.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/index_auto_self_join.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/index_auto_self_join.q.out (working copy) @@ -16,7 +16,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/innerjoin.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/innerjoin.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/innerjoin.q.out (working copy) @@ -27,7 +27,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/input14.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/input14.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/input14.q.out (working copy) @@ -33,7 +33,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/input17.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/input17.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/input17.q.out (working copy) @@ -33,7 +33,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/input18.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/input18.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/input18.q.out (working copy) @@ -33,7 +33,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join0.q.java1.7.out =================================================================== --- ql/src/test/results/clientpositive/spark/join0.q.java1.7.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join0.q.java1.7.out (working copy) @@ -30,7 +30,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join1.q.out (working copy) @@ -27,7 +27,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join10.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join10.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join10.q.out (working copy) @@ -24,7 +24,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join11.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join11.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join11.q.out (working copy) @@ -26,7 +26,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join12.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join12.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join12.q.out (working copy) @@ -32,8 +32,8 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 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 Index: ql/src/test/results/clientpositive/spark/join13.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join13.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join13.q.out (working copy) @@ -32,8 +32,8 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 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 Index: ql/src/test/results/clientpositive/spark/join14.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join14.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join14.q.out (working copy) @@ -29,7 +29,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join15.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join15.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join15.q.out (working copy) @@ -16,8 +16,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join16.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join16.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join16.q.out (working copy) @@ -10,7 +10,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join17.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join17.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join17.q.out (working copy) @@ -65,7 +65,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join18.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join18.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join18.q.out (working copy) @@ -36,9 +36,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out (working copy) @@ -38,9 +38,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join19.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join19.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join19.q.out (working copy) @@ -130,9 +130,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 9 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 2), Map 8 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 9 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join2.q.out (working copy) @@ -27,8 +27,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join20.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join20.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join20.q.out (working copy) @@ -18,8 +18,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join21.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join21.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join21.q.out (working copy) @@ -16,8 +16,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join22.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join22.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join22.q.out (working copy) @@ -12,8 +12,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join23.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join23.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join23.q.out (working copy) @@ -18,7 +18,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join29.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join29.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join29.q.out (working copy) @@ -36,7 +36,7 @@ Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -77,7 +77,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/join3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join3.q.out (working copy) @@ -27,7 +27,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join30.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join30.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join30.q.out (working copy) @@ -47,7 +47,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP, 1) + Reducer 3 <- Map 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 2 Index: ql/src/test/results/clientpositive/spark/join31.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join31.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join31.q.out (working copy) @@ -38,7 +38,7 @@ Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -83,8 +83,8 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (GROUP, 1) - Reducer 5 <- Reducer 4 (GROUP, 1) + Reducer 4 <- Map 3 (GROUP, 2) + Reducer 5 <- Reducer 4 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/join34.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join34.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join34.q.out (working copy) @@ -147,7 +147,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join35.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join35.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join35.q.out (working copy) @@ -155,9 +155,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join38.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join38.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join38.q.out (working copy) @@ -76,7 +76,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP, 3) + Reducer 3 <- Map 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 2 Index: ql/src/test/results/clientpositive/spark/join4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join4.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join4.q.out (working copy) @@ -49,7 +49,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join40.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join40.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join40.q.out (working copy) @@ -16,7 +16,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join5.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join5.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join5.q.out (working copy) @@ -49,7 +49,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join6.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join6.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join6.q.out (working copy) @@ -49,7 +49,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join7.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join7.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join7.q.out (working copy) @@ -59,7 +59,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join8.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join8.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join8.q.out (working copy) @@ -49,7 +49,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join9.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join9.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join9.q.out (working copy) @@ -81,7 +81,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out (working copy) @@ -78,7 +78,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -178,7 +178,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -280,7 +280,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -398,9 +398,9 @@ Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (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 @@ -550,9 +550,9 @@ Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (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 Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out (working copy) @@ -12,7 +12,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -113,7 +113,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -216,7 +216,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -328,7 +328,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out (working copy) @@ -12,8 +12,8 @@ Stage: Stage-1 Spark Edges: - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -150,9 +150,9 @@ Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (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 Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out (working copy) @@ -14,7 +14,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -117,7 +117,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -222,7 +222,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -336,7 +336,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out (working copy) @@ -14,8 +14,8 @@ Stage: Stage-1 Spark Edges: - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -154,9 +154,9 @@ Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (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 Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual1.q.out (working copy) @@ -68,7 +68,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -157,7 +157,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -248,7 +248,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -349,7 +349,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual2.q.out (working copy) @@ -68,7 +68,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 @@ -175,9 +175,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual3.q.out (working copy) @@ -70,7 +70,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -164,7 +164,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -260,7 +260,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -366,7 +366,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual4.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_cond_pushdown_unqual4.q.out (working copy) @@ -70,7 +70,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 @@ -182,9 +182,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join_hive_626.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_hive_626.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_hive_626.q.out (working copy) @@ -72,8 +72,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join_merge_multi_expressions.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_merge_multi_expressions.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_merge_multi_expressions.q.out (working copy) @@ -12,7 +12,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/join_merging.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_merging.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_merging.q.out (working copy) @@ -16,8 +16,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + 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 @@ -133,8 +133,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join_nullsafe.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_nullsafe.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_nullsafe.q.out (working copy) @@ -32,7 +32,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -117,7 +117,7 @@ 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 @@ -214,7 +214,7 @@ 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 @@ -330,7 +330,7 @@ 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 @@ -425,7 +425,7 @@ 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 @@ -1564,7 +1564,7 @@ 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 @@ -1578,7 +1578,7 @@ Reduce Output Operator key expressions: null (type: void) sort order: + - Map-reduce partition columns: '' (type: string) + Map-reduce partition columns: null (type: void) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: int) Map 3 @@ -1592,7 +1592,7 @@ Reduce Output Operator key expressions: null (type: void) sort order: + - Map-reduce partition columns: '' (type: string) + Map-reduce partition columns: null (type: void) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Reducer 2 Index: ql/src/test/results/clientpositive/spark/join_reorder.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_reorder.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_reorder.q.out (working copy) @@ -64,7 +64,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -137,7 +137,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join_reorder2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_reorder2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_reorder2.q.out (working copy) @@ -82,7 +82,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join_reorder3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_reorder3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_reorder3.q.out (working copy) @@ -82,7 +82,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/join_thrift.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_thrift.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_thrift.q.out (working copy) @@ -38,7 +38,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join_vc.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_vc.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_vc.q.out (working copy) @@ -18,8 +18,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 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: Index: ql/src/test/results/clientpositive/spark/join_view.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join_view.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/join_view.q.out (working copy) @@ -50,7 +50,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out (working copy) @@ -545,7 +545,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out (working copy) @@ -112,7 +112,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/mapjoin_distinct.q.out (working copy) @@ -38,8 +38,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out (working copy) @@ -637,7 +637,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/mapreduce1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mapreduce1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/mapreduce1.q.out (working copy) @@ -31,7 +31,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/mapreduce2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mapreduce2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/mapreduce2.q.out (working copy) @@ -29,7 +29,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/merge1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/merge1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/merge1.q.out (working copy) @@ -32,7 +32,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/merge2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/merge2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/merge2.q.out (working copy) @@ -32,7 +32,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/mergejoins.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mergejoins.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/mergejoins.q.out (working copy) @@ -50,8 +50,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -186,7 +186,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/mergejoins_mixed.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mergejoins_mixed.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/mergejoins_mixed.q.out (working copy) @@ -26,7 +26,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 @@ -119,7 +119,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 @@ -212,7 +212,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 @@ -305,7 +305,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) + 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 @@ -400,8 +400,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (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 @@ -508,8 +508,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (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 @@ -616,8 +616,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (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 @@ -726,9 +726,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + 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 @@ -855,9 +855,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + 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 @@ -984,9 +984,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + 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 @@ -1113,9 +1113,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + 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 @@ -1244,8 +1244,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out (working copy) @@ -43,7 +43,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out (working copy) @@ -50,7 +50,7 @@ Spark Edges: Reducer 2 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -172,7 +172,7 @@ Spark Edges: Reducer 2 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out (working copy) @@ -53,8 +53,8 @@ Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 6 (GROUP, 1) - Reducer 4 <- Map 7 (GROUP, 1) + Reducer 2 <- Map 6 (GROUP, 2) + Reducer 4 <- Map 7 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) Reducer 5 <- Reducer 4 (SORT, 1) #### A masked pattern was here #### Index: ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out (working copy) @@ -59,8 +59,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP, 1) - Reducer 3 <- Map 5 (GROUP, 1) + Reducer 2 <- Map 4 (GROUP, 2) + Reducer 3 <- Map 5 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -193,8 +193,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP, 1) - Reducer 3 <- Map 5 (GROUP, 1) + Reducer 2 <- Map 4 (GROUP, 2) + Reducer 3 <- Map 5 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -327,8 +327,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 4 (GROUP, 1) - Reducer 3 <- Map 5 (GROUP, 1) + Reducer 2 <- Map 4 (GROUP, 2) + Reducer 3 <- Map 5 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -461,7 +461,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -571,9 +571,9 @@ Stage: Stage-3 Spark Edges: - Reducer 2 <- Map 5 (GROUP, 1) - Reducer 3 <- Map 6 (GROUP, 1) - Reducer 4 <- Map 7 (GROUP, 1) + Reducer 2 <- Map 5 (GROUP, 2) + Reducer 3 <- Map 6 (GROUP, 2) + Reducer 4 <- Map 7 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 5 Index: ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out (working copy) @@ -103,7 +103,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/orc_analyze.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/orc_analyze.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/orc_analyze.q.out (working copy) @@ -107,7 +107,7 @@ numFiles 1 numRows 100 rawDataSize 52600 - totalSize 3123 + totalSize 3158 #### A masked pattern was here #### # Storage Information @@ -197,7 +197,7 @@ numFiles 1 numRows 100 rawDataSize 52600 - totalSize 3123 + totalSize 3158 #### A masked pattern was here #### # Storage Information @@ -313,7 +313,7 @@ numFiles 1 numRows 50 rawDataSize 21950 - totalSize 2026 + totalSize 2055 #### A masked pattern was here #### # Storage Information @@ -358,7 +358,7 @@ numFiles 1 numRows 50 rawDataSize 22050 - totalSize 2045 + totalSize 2071 #### A masked pattern was here #### # Storage Information @@ -460,7 +460,7 @@ numFiles 1 numRows 50 rawDataSize 21950 - totalSize 2026 + totalSize 2055 #### A masked pattern was here #### # Storage Information @@ -505,7 +505,7 @@ numFiles 1 numRows 50 rawDataSize 22050 - totalSize 2045 + totalSize 2071 #### A masked pattern was here #### # Storage Information @@ -627,7 +627,7 @@ numFiles 1 numRows 50 rawDataSize 21950 - totalSize 2026 + totalSize 2055 #### A masked pattern was here #### # Storage Information @@ -672,7 +672,7 @@ numFiles 1 numRows 50 rawDataSize 22050 - totalSize 2045 + totalSize 2071 #### A masked pattern was here #### # Storage Information @@ -780,7 +780,7 @@ numFiles 1 numRows 50 rawDataSize 21950 - totalSize 2026 + totalSize 2055 #### A masked pattern was here #### # Storage Information @@ -825,7 +825,7 @@ numFiles 1 numRows 50 rawDataSize 22050 - totalSize 2045 + totalSize 2071 #### A masked pattern was here #### # Storage Information @@ -992,7 +992,7 @@ numFiles 1 numRows 50 rawDataSize 21950 - totalSize 2026 + totalSize 2055 #### A masked pattern was here #### # Storage Information Index: ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out =================================================================== --- ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out (working copy) @@ -114,7 +114,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/parallel.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/parallel.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/parallel.q.out (working copy) @@ -39,8 +39,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ppd_gby_join.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_gby_join.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_gby_join.q.out (working copy) @@ -30,8 +30,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ppd_join.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_join.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_join.q.out (working copy) @@ -28,7 +28,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ppd_join2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_join2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_join2.q.out (working copy) @@ -34,8 +34,8 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 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 Index: ql/src/test/results/clientpositive/spark/ppd_join3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_join3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_join3.q.out (working copy) @@ -34,8 +34,8 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 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 Index: ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out (working copy) @@ -127,8 +127,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out (working copy) @@ -54,7 +54,7 @@ Stage: Stage-4 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -821,29 +821,16 @@ 298 2008-04-08 12 298 2008-04-08 12 298 2008-04-08 12 -val_302 val_305 -val_306 val_307 val_307 val_307 val_307 -val_308 val_309 val_309 val_309 val_309 val_310 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_315 val_316 val_316 val_316 @@ -853,10 +840,6 @@ val_316 val_316 val_316 -val_317 -val_317 -val_317 -val_317 val_318 val_318 val_318 @@ -870,10 +853,6 @@ val_321 val_321 val_321 -val_322 -val_322 -val_322 -val_322 val_323 val_325 val_325 @@ -888,20 +867,185 @@ val_327 val_327 val_327 +val_332 +val_336 +val_338 +val_341 +val_345 +val_356 +val_365 +val_367 +val_367 +val_367 +val_367 +val_369 +val_369 +val_369 +val_369 +val_369 +val_369 +val_369 +val_369 +val_369 +val_374 +val_378 +val_389 +val_392 +val_394 +val_396 +val_396 +val_396 +val_396 +val_396 +val_396 +val_396 +val_396 +val_396 +val_400 +val_402 +val_404 +val_404 +val_404 +val_404 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_411 +val_413 +val_413 +val_413 +val_413 +val_417 +val_417 +val_417 +val_417 +val_417 +val_417 +val_417 +val_417 +val_417 +val_419 +val_424 +val_424 +val_424 +val_424 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 +val_435 +val_437 +val_439 +val_439 +val_439 +val_439 +val_444 +val_446 +val_448 +val_453 +val_455 +val_457 +val_459 +val_459 +val_459 +val_459 +val_460 +val_462 +val_462 +val_462 +val_462 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_468 +val_475 +val_477 +val_479 +val_480 +val_480 +val_480 +val_480 +val_480 +val_480 +val_480 +val_480 +val_480 +val_482 +val_484 +val_491 +val_493 +val_495 +val_497 +val_302 +val_306 +val_308 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_315 +val_317 +val_317 +val_317 +val_317 +val_322 +val_322 +val_322 +val_322 val_331 val_331 val_331 val_331 -val_332 val_333 val_333 val_333 val_333 val_335 -val_336 -val_338 val_339 -val_341 val_342 val_342 val_342 @@ -910,7 +1054,6 @@ val_344 val_344 val_344 -val_345 val_348 val_348 val_348 @@ -941,31 +1084,14 @@ val_353 val_353 val_353 -val_356 val_360 val_362 val_364 -val_365 val_366 -val_367 -val_367 -val_367 -val_367 val_368 -val_369 -val_369 -val_369 -val_369 -val_369 -val_369 -val_369 -val_369 -val_369 val_373 -val_374 val_375 val_377 -val_378 val_379 val_382 val_382 @@ -981,23 +1107,11 @@ val_384 val_384 val_386 -val_389 -val_392 val_393 -val_394 val_395 val_395 val_395 val_395 -val_396 -val_396 -val_396 -val_396 -val_396 -val_396 -val_396 -val_396 -val_396 val_397 val_397 val_397 @@ -1006,7 +1120,6 @@ val_399 val_399 val_399 -val_400 val_401 val_401 val_401 @@ -1032,7 +1145,6 @@ val_401 val_401 val_401 -val_402 val_403 val_403 val_403 @@ -1042,26 +1154,6 @@ val_403 val_403 val_403 -val_404 -val_404 -val_404 -val_404 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 val_407 val_409 val_409 @@ -1072,31 +1164,12 @@ val_409 val_409 val_409 -val_411 -val_413 -val_413 -val_413 -val_413 val_414 val_414 val_414 val_414 -val_417 -val_417 -val_417 -val_417 -val_417 -val_417 -val_417 -val_417 -val_417 val_418 -val_419 val_421 -val_424 -val_424 -val_424 -val_424 val_427 val_429 val_429 @@ -1111,19 +1184,8 @@ val_430 val_430 val_430 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 val_432 -val_435 val_436 -val_437 val_438 val_438 val_438 @@ -1133,17 +1195,9 @@ val_438 val_438 val_438 -val_439 -val_439 -val_439 -val_439 val_443 -val_444 -val_446 -val_448 val_449 val_452 -val_453 val_454 val_454 val_454 @@ -1153,51 +1207,15 @@ val_454 val_454 val_454 -val_455 -val_457 val_458 val_458 val_458 val_458 -val_459 -val_459 -val_459 -val_459 -val_460 -val_462 -val_462 -val_462 -val_462 val_463 val_463 val_463 val_463 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 val_467 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 -val_468 val_469 val_469 val_469 @@ -1225,26 +1243,12 @@ val_469 val_470 val_472 -val_475 -val_477 val_478 val_478 val_478 val_478 -val_479 -val_480 -val_480 -val_480 -val_480 -val_480 -val_480 -val_480 -val_480 -val_480 val_481 -val_482 val_483 -val_484 val_485 val_487 val_489 @@ -1264,16 +1268,12 @@ val_489 val_489 val_490 -val_491 val_492 val_492 val_492 val_492 -val_493 val_494 -val_495 val_496 -val_497 val_498 val_498 val_498 @@ -1311,7 +1311,7 @@ Stage: Stage-4 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 @@ -2078,12 +2078,25 @@ 298 2008-04-08 12 298 2008-04-08 12 298 2008-04-08 12 -val_306 +val_305 +val_307 +val_307 +val_307 +val_307 val_309 val_309 val_309 val_309 -val_315 +val_310 +val_316 +val_316 +val_316 +val_316 +val_316 +val_316 +val_316 +val_316 +val_316 val_318 val_318 val_318 @@ -2097,6 +2110,11 @@ val_321 val_321 val_321 +val_323 +val_325 +val_325 +val_325 +val_325 val_327 val_327 val_327 @@ -2106,45 +2124,17 @@ val_327 val_327 val_327 -val_333 -val_333 -val_333 -val_333 +val_332 val_336 -val_339 -val_342 -val_342 -val_342 -val_342 +val_338 +val_341 val_345 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_348 -val_351 -val_360 -val_366 +val_356 +val_365 +val_367 +val_367 +val_367 +val_367 val_369 val_369 val_369 @@ -2154,18 +2144,11 @@ val_369 val_369 val_369 -val_375 +val_374 val_378 -val_384 -val_384 -val_384 -val_384 -val_384 -val_384 -val_384 -val_384 -val_384 -val_393 +val_389 +val_392 +val_394 val_396 val_396 val_396 @@ -2175,16 +2158,33 @@ val_396 val_396 val_396 -val_399 -val_399 -val_399 -val_399 +val_400 val_402 +val_404 +val_404 +val_404 +val_404 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 +val_406 val_411 -val_414 -val_414 -val_414 -val_414 +val_413 +val_413 +val_413 +val_413 val_417 val_417 val_417 @@ -2194,31 +2194,50 @@ val_417 val_417 val_417 -val_429 -val_429 -val_429 -val_429 -val_432 +val_419 +val_424 +val_424 +val_424 +val_424 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 +val_431 val_435 -val_438 -val_438 -val_438 -val_438 -val_438 -val_438 -val_438 -val_438 -val_438 +val_437 +val_439 +val_439 +val_439 +val_439 val_444 +val_446 +val_448 val_453 +val_455 +val_457 val_459 val_459 val_459 val_459 +val_460 val_462 val_462 val_462 val_462 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 +val_466 val_468 val_468 val_468 @@ -2235,7 +2254,9 @@ val_468 val_468 val_468 +val_475 val_477 +val_479 val_480 val_480 val_480 @@ -2245,80 +2266,142 @@ val_480 val_480 val_480 -val_483 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_489 -val_492 -val_492 -val_492 -val_492 +val_482 +val_484 +val_491 +val_493 val_495 -val_498 -val_498 -val_498 -val_498 -val_498 -val_498 -val_498 -val_498 -val_498 -val_307 -val_307 -val_307 -val_307 -val_310 -val_316 -val_316 -val_316 -val_316 -val_316 -val_316 -val_316 -val_316 -val_316 +val_497 +val_302 +val_306 +val_308 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_311 +val_315 +val_317 +val_317 +val_317 +val_317 val_322 val_322 val_322 val_322 -val_325 -val_325 -val_325 -val_325 val_331 val_331 val_331 val_331 +val_333 +val_333 +val_333 +val_333 +val_335 +val_339 +val_342 +val_342 +val_342 +val_342 +val_344 +val_344 +val_344 +val_344 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_348 +val_351 +val_353 +val_353 +val_353 +val_353 +val_360 +val_362 val_364 -val_367 -val_367 -val_367 -val_367 +val_366 +val_368 val_373 +val_375 +val_377 val_379 val_382 val_382 val_382 val_382 -val_394 +val_384 +val_384 +val_384 +val_384 +val_384 +val_384 +val_384 +val_384 +val_384 +val_386 +val_393 +val_395 +val_395 +val_395 +val_395 val_397 val_397 val_397 val_397 -val_400 +val_399 +val_399 +val_399 +val_399 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 +val_401 val_403 val_403 val_403 @@ -2328,22 +2411,7 @@ val_403 val_403 val_403 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 -val_406 +val_407 val_409 val_409 val_409 @@ -2353,13 +2421,17 @@ val_409 val_409 val_409 +val_414 +val_414 +val_414 +val_414 val_418 val_421 -val_424 -val_424 -val_424 -val_424 val_427 +val_429 +val_429 +val_429 +val_429 val_430 val_430 val_430 @@ -2369,12 +2441,20 @@ val_430 val_430 val_430 +val_432 val_436 -val_439 -val_439 -val_439 -val_439 -val_448 +val_438 +val_438 +val_438 +val_438 +val_438 +val_438 +val_438 +val_438 +val_438 +val_443 +val_449 +val_452 val_454 val_454 val_454 @@ -2384,21 +2464,15 @@ val_454 val_454 val_454 -val_457 -val_460 +val_458 +val_458 +val_458 +val_458 val_463 val_463 val_463 val_463 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 -val_466 +val_467 val_469 val_469 val_469 @@ -2424,119 +2498,45 @@ val_469 val_469 val_469 +val_470 val_472 -val_475 val_478 val_478 val_478 val_478 val_481 -val_484 +val_483 +val_485 val_487 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 +val_489 val_490 -val_493 +val_492 +val_492 +val_492 +val_492 +val_494 val_496 -val_302 -val_305 -val_308 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_311 -val_317 -val_317 -val_317 -val_317 -val_323 -val_332 -val_335 -val_338 -val_341 -val_344 -val_344 -val_344 -val_344 -val_353 -val_353 -val_353 -val_353 -val_356 -val_362 -val_365 -val_368 -val_374 -val_377 -val_386 -val_389 -val_392 -val_395 -val_395 -val_395 -val_395 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_401 -val_404 -val_404 -val_404 -val_404 -val_407 -val_413 -val_413 -val_413 -val_413 -val_419 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 -val_431 -val_437 -val_443 -val_446 -val_449 -val_452 -val_455 -val_458 -val_458 -val_458 -val_458 -val_467 -val_470 -val_479 -val_482 -val_485 -val_491 -val_494 -val_497 +val_498 +val_498 +val_498 +val_498 +val_498 +val_498 +val_498 +val_498 +val_498 Index: ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_outer_join1.q.out (working copy) @@ -28,7 +28,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_outer_join2.q.out (working copy) @@ -28,7 +28,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_outer_join3.q.out (working copy) @@ -28,7 +28,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out (working copy) @@ -34,8 +34,8 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 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 Index: ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out (working copy) @@ -42,7 +42,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -129,7 +129,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 @@ -216,7 +216,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/ppd_transform.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_transform.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ppd_transform.q.out (working copy) @@ -28,7 +28,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ptf.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ptf.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ptf.q.out (working copy) @@ -32,8 +32,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ptf_streaming.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ptf_streaming.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/ptf_streaming.q.out (working copy) @@ -32,8 +32,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out (working copy) @@ -30,7 +30,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/router_join_ppr.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/router_join_ppr.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/router_join_ppr.q.out (working copy) @@ -112,7 +112,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/sample8.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/sample8.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/sample8.q.out (working copy) @@ -92,7 +92,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoin.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoin.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoin.q.out (working copy) @@ -90,7 +90,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out (working copy) @@ -18,7 +18,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoin_union_remove_1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoin_union_remove_1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoin_union_remove_1.q.out (working copy) @@ -68,8 +68,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoin_union_remove_2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoin_union_remove_2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoin_union_remove_2.q.out (working copy) @@ -80,8 +80,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2), Map 8 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt1.q.out (working copy) @@ -54,8 +54,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt11.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt11.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt11.q.out (working copy) @@ -66,10 +66,10 @@ Stage: Stage-1 Spark Edges: - Reducer 11 <- Map 10 (PARTITION-LEVEL SORT, 1), Map 12 (PARTITION-LEVEL SORT, 1) - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 8 <- Map 7 (PARTITION-LEVEL SORT, 1), Map 9 (PARTITION-LEVEL SORT, 1) + Reducer 11 <- Map 10 (PARTITION-LEVEL SORT, 2), Map 12 (PARTITION-LEVEL SORT, 2) + 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) + Reducer 8 <- Map 7 (PARTITION-LEVEL SORT, 2), Map 9 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoinopt12.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt12.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt12.q.out (working copy) @@ -56,8 +56,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt13.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt13.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt13.q.out (working copy) @@ -82,8 +82,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt14.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt14.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt14.q.out (working copy) @@ -86,9 +86,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 2), Map 8 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoinopt16.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt16.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt16.q.out (working copy) @@ -56,8 +56,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt17.q.out (working copy) @@ -60,8 +60,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt19.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt19.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt19.q.out (working copy) @@ -58,8 +58,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt2.q.out (working copy) @@ -64,8 +64,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt20.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt20.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt20.q.out (working copy) @@ -58,8 +58,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt3.q.out (working copy) @@ -58,8 +58,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt4.q.out (working copy) @@ -54,8 +54,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt5.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt5.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt5.q.out (working copy) @@ -56,8 +56,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt6.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt6.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt6.q.out (working copy) @@ -58,8 +58,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/skewjoinopt7.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt7.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt7.q.out (working copy) @@ -74,8 +74,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2), Map 8 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoinopt8.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt8.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt8.q.out (working copy) @@ -72,8 +72,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1), Map 8 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2), Map 8 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/skewjoinopt9.q.out (working copy) @@ -64,7 +64,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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 Index: ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out (working copy) @@ -60,9 +60,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 1), Map 7 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + 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 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/sort.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/sort.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/sort.q.out (working copy) @@ -16,7 +16,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/subquery_exists.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/subquery_exists.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/subquery_exists.q.out (working copy) @@ -30,7 +30,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/subquery_in.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/subquery_in.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/subquery_in.q.out (working copy) @@ -22,7 +22,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -885,3 +885,5 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@lineitem #### A masked pattern was here #### +108570 8571 +4297 1798 Index: ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out =================================================================== --- ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out (working copy) @@ -75,8 +75,8 @@ Spark Edges: Reducer 2 <- Map 10 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) - Reducer 5 <- Map 11 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 11 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) Reducer 9 <- Map 8 (GROUP, 1) Reducer 4 <- Reducer 3 (SORT, 1) #### A masked pattern was here #### Index: ql/src/test/results/clientpositive/spark/tez_join_tests.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/tez_join_tests.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/tez_join_tests.q.out (working copy) @@ -16,8 +16,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) Reducer 5 <- Reducer 4 (SORT, 1) #### A masked pattern was here #### Index: ql/src/test/results/clientpositive/spark/tez_joins_explain.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/tez_joins_explain.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/tez_joins_explain.q.out (working copy) @@ -16,8 +16,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 6 (PARTITION-LEVEL SORT, 1) - Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (SORT, 1) Reducer 5 <- Reducer 4 (SORT, 1) #### A masked pattern was here #### Index: ql/src/test/results/clientpositive/spark/transform_ppr1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/transform_ppr1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/transform_ppr1.q.out (working copy) @@ -103,7 +103,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/transform_ppr2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/transform_ppr2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/transform_ppr2.q.out (working copy) @@ -105,7 +105,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union11.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union11.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union11.q.out (working copy) @@ -29,7 +29,7 @@ Reducer 2 <- Map 1 (GROUP, 1) Reducer 5 <- Map 4 (GROUP, 1) Reducer 7 <- Map 6 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP, 1), Reducer 5 (GROUP, 1), Reducer 7 (GROUP, 1) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 5 (GROUP, 2), Reducer 7 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union14.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union14.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union14.q.out (working copy) @@ -25,7 +25,7 @@ Spark Edges: Reducer 4 <- Map 3 (GROUP, 1) - Reducer 2 <- Map 1 (GROUP, 1), Reducer 4 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2), Reducer 4 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -128,20 +128,20 @@ POSTHOOK: Input: default@src POSTHOOK: Input: default@src1 #### A masked pattern was here #### +238 1 +311 1 +146 1 +278 1 +401 1 +128 1 +98 1 255 1 213 1 -238 1 150 1 -311 1 -146 1 369 1 66 1 10 -278 1 -401 1 406 1 tst1 1 -128 1 -98 1 273 1 224 1 Index: ql/src/test/results/clientpositive/spark/union15.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union15.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union15.q.out (working copy) @@ -27,7 +27,7 @@ Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 4 (GROUP, 1), Map 5 (GROUP, 1), Reducer 2 (GROUP, 1) + Reducer 3 <- Map 4 (GROUP, 2), Map 5 (GROUP, 2), Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -152,20 +152,20 @@ POSTHOOK: Input: default@src POSTHOOK: Input: default@src1 #### A masked pattern was here #### +238 2 +311 2 +146 2 +278 2 +401 2 +128 2 +98 2 255 2 213 2 -238 2 150 2 -311 2 -146 2 369 2 66 2 20 -278 2 -401 2 406 2 tst1 1 -128 2 -98 2 273 2 224 2 Index: ql/src/test/results/clientpositive/spark/union19.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union19.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union19.q.out (working copy) @@ -48,7 +48,7 @@ Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 4 (GROUP, 1), Reducer 2 (GROUP, 1) + Reducer 3 <- Map 4 (GROUP, 2), Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union28.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union28.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union28.q.out (working copy) @@ -41,8 +41,8 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP, 1) - Reducer 5 <- Map 4 (GROUP, 1) + Reducer 3 <- Map 2 (GROUP, 2) + Reducer 5 <- Map 4 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union3.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union3.q.out (working copy) @@ -50,8 +50,8 @@ Reducer 2 <- Map 1 (GROUP, 1) Reducer 5 <- Map 4 (GROUP, 1) Reducer 8 <- Map 7 (GROUP, 1) - Reducer 3 <- Reducer 2 (SORT, 1) - Reducer 6 <- Reducer 5 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 6 <- Reducer 5 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -250,7 +250,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@union_out #### A masked pattern was here #### +4 1 2 3 -4 Index: ql/src/test/results/clientpositive/spark/union30.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union30.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union30.q.out (working copy) @@ -55,8 +55,8 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP, 1) - Reducer 5 <- Map 4 (GROUP, 1) + Reducer 3 <- Map 2 (GROUP, 2) + Reducer 5 <- Map 4 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union33.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union33.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union33.q.out (working copy) @@ -41,8 +41,8 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 4 <- Reducer 3 (GROUP, 1) + Reducer 3 <- Map 2 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union5.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union5.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union5.q.out (working copy) @@ -24,7 +24,7 @@ Edges: Reducer 2 <- Map 1 (GROUP, 1) Reducer 5 <- Map 4 (GROUP, 1) - Reducer 3 <- Reducer 2 (GROUP, 1), Reducer 5 (GROUP, 1) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 5 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union7.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union7.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union7.q.out (working copy) @@ -23,7 +23,7 @@ Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 3 <- Map 4 (GROUP, 1), Reducer 2 (GROUP, 1) + Reducer 3 <- Map 4 (GROUP, 2), Reducer 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -124,20 +124,20 @@ POSTHOOK: Input: default@src POSTHOOK: Input: default@src1 #### A masked pattern was here #### +238 1 +311 1 +146 1 +278 1 +401 1 +128 1 +98 1 255 1 213 1 -238 1 150 1 -311 1 -146 1 369 1 66 1 10 -278 1 -401 1 406 1 tst1 1 -128 1 -98 1 273 1 224 1 Index: ql/src/test/results/clientpositive/spark/union_ppr.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_ppr.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_ppr.q.out (working copy) @@ -116,7 +116,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1), Map 3 (SORT, 1) + Reducer 2 <- Map 1 (SORT, 2), Map 3 (SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union_remove_1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_1.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_1.q.out (working copy) @@ -68,8 +68,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -195,7 +195,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 totalSize 40 Index: ql/src/test/results/clientpositive/spark/union_remove_10.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_10.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_10.q.out (working copy) @@ -89,7 +89,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (GROUP, 1) + Reducer 3 <- Map 2 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -252,10 +252,10 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 3 + numFiles 4 numRows -1 rawDataSize -1 - totalSize 271 + totalSize 350 #### A masked pattern was here #### # Storage Information Index: ql/src/test/results/clientpositive/spark/union_remove_15.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_15.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_15.q.out (working copy) @@ -74,8 +74,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union_remove_16.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_16.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_16.q.out (working copy) @@ -77,8 +77,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union_remove_18.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_18.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_18.q.out (working copy) @@ -72,8 +72,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union_remove_19.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_19.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_19.q.out (working copy) @@ -72,8 +72,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -199,7 +199,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 totalSize 40 Index: ql/src/test/results/clientpositive/spark/union_remove_2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_2.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_2.q.out (working copy) @@ -74,7 +74,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -197,7 +197,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 3 + numFiles 4 numRows -1 rawDataSize -1 totalSize 68 Index: ql/src/test/results/clientpositive/spark/union_remove_20.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_20.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_20.q.out (working copy) @@ -70,8 +70,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -203,7 +203,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 totalSize 40 Index: ql/src/test/results/clientpositive/spark/union_remove_21.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_21.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_21.q.out (working copy) @@ -70,8 +70,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -207,7 +207,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 totalSize 20 Index: ql/src/test/results/clientpositive/spark/union_remove_24.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_24.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_24.q.out (working copy) @@ -66,8 +66,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -199,7 +199,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 totalSize 60 Index: ql/src/test/results/clientpositive/spark/union_remove_25.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_25.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_25.q.out (working copy) @@ -84,8 +84,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -218,7 +218,7 @@ #### A masked pattern was here #### Partition Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 totalSize 40 Index: ql/src/test/results/clientpositive/spark/union_remove_4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_4.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_4.q.out (working copy) @@ -73,8 +73,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -245,7 +245,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 totalSize 40 Index: ql/src/test/results/clientpositive/spark/union_remove_5.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_5.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_5.q.out (working copy) @@ -81,7 +81,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -249,7 +249,7 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 3 + numFiles 4 numRows -1 rawDataSize -1 totalSize 68 Index: ql/src/test/results/clientpositive/spark/union_remove_6.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_6.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_6.q.out (working copy) @@ -71,8 +71,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union_remove_7.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_7.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_7.q.out (working copy) @@ -72,8 +72,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 3 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -199,10 +199,10 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 2 + numFiles 4 numRows -1 rawDataSize -1 - totalSize 178 + totalSize 336 #### A masked pattern was here #### # Storage Information Index: ql/src/test/results/clientpositive/spark/union_remove_8.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_8.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_8.q.out (working copy) @@ -78,7 +78,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -201,10 +201,10 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 3 + numFiles 4 numRows -1 rawDataSize -1 - totalSize 271 + totalSize 350 #### A masked pattern was here #### # Storage Information Index: ql/src/test/results/clientpositive/spark/union_remove_9.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_9.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/union_remove_9.q.out (working copy) @@ -85,7 +85,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -255,10 +255,10 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE false - numFiles 3 + numFiles 4 numRows -1 rawDataSize -1 - totalSize 271 + totalSize 350 #### A masked pattern was here #### # Storage Information Index: ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out (working copy) @@ -73,7 +73,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/vectorization_13.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vectorization_13.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/vectorization_13.q.out (working copy) @@ -72,7 +72,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/vectorization_14.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vectorization_14.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/vectorization_14.q.out (working copy) @@ -72,7 +72,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/vectorization_15.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vectorization_15.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/vectorization_15.q.out (working copy) @@ -68,7 +68,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) Reducer 3 <- Reducer 2 (SORT, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/vectorization_16.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vectorization_16.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/vectorization_16.q.out (working copy) @@ -50,7 +50,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/vectorization_9.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vectorization_9.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/vectorization_9.q.out (working copy) @@ -46,7 +46,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out (revision 1654728) +++ ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out (working copy) @@ -14,7 +14,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + 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: Index: spark-client/pom.xml =================================================================== --- spark-client/pom.xml (revision 1654728) +++ spark-client/pom.xml (working copy) @@ -65,6 +65,11 @@ junit test + + org.mockito + mockito-all + test + Index: spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java (working copy) @@ -121,6 +121,20 @@ } + protected static class JobStarted implements Serializable { + + final String id; + + JobStarted(String id) { + this.id = id; + } + + JobStarted() { + this(null); + } + + } + /** * Inform the client that a new spark job has been submitted for the client job. */ @@ -138,4 +152,18 @@ } } + protected static class SyncJobRequest implements Serializable { + + final Job job; + + SyncJobRequest(Job job) { + this.job = job; + } + + SyncJobRequest() { + this(null); + } + + } + } Index: spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java (working copy) @@ -55,4 +55,53 @@ */ SparkCounters getSparkCounters(); + /** + * Return the current state of the job. + */ + State getState(); + + /** + * Add a listener to the job handle. If the job's state is not SENT, a callback for the + * corresponding state will be invoked immediately. + * + * @param l The listener to add. + */ + void addListener(Listener l); + + /** + * The current state of the submitted job. + */ + static enum State { + SENT, + QUEUED, + STARTED, + CANCELLED, + FAILED, + SUCCEEDED; + } + + /** + * A listener for monitoring the state of the job in the remote context. Callbacks are called + * when the corresponding state change occurs. + */ + static interface Listener { + + void onJobQueued(JobHandle job); + + void onJobStarted(JobHandle job); + + void onJobCancelled(JobHandle job); + + void onJobFailed(JobHandle job, Throwable cause); + + void onJobSucceeded(JobHandle job, T result); + + /** + * Called when a monitored Spark job is started on the remote context. This callback + * does not indicate a state change in the client job's status. + */ + void onSparkJobStarted(JobHandle job, int sparkJobId); + + } + } Index: spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java (working copy) @@ -17,16 +17,17 @@ package org.apache.hive.spark.client; -import io.netty.util.concurrent.Promise; - import java.io.Serializable; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import io.netty.util.concurrent.Promise; + import org.apache.hive.spark.counter.SparkCounters; /** @@ -34,28 +35,30 @@ */ class JobHandleImpl implements JobHandle { - private final AtomicBoolean cancelled; private final SparkClientImpl client; private final String jobId; private final MetricsCollection metrics; private final Promise promise; private final List sparkJobIds; + private final List listeners; + private volatile State state; private volatile SparkCounters sparkCounters; JobHandleImpl(SparkClientImpl client, Promise promise, String jobId) { - this.cancelled = new AtomicBoolean(); this.client = client; this.jobId = jobId; this.promise = promise; + this.listeners = Lists.newLinkedList(); this.metrics = new MetricsCollection(); this.sparkJobIds = new CopyOnWriteArrayList(); + this.state = State.SENT; this.sparkCounters = null; } /** Requests a running job to be cancelled. */ @Override public boolean cancel(boolean mayInterrupt) { - if (cancelled.compareAndSet(false, true)) { + if (changeState(State.CANCELLED)) { client.cancel(jobId); promise.cancel(mayInterrupt); return true; @@ -114,20 +117,116 @@ return sparkCounters; } + @Override + public State getState() { + return state; + } + + @Override + public void addListener(Listener l) { + synchronized (listeners) { + listeners.add(l); + // If current state is a final state, notify of Spark job IDs before notifying about the + // state transition. + if (state.ordinal() >= State.CANCELLED.ordinal()) { + for (Integer i : sparkJobIds) { + l.onSparkJobStarted(this, i); + } + } + + fireStateChange(state, l); + + // Otherwise, notify about Spark jobs after the state notification. + if (state.ordinal() < State.CANCELLED.ordinal()) { + for (Integer i : sparkJobIds) { + l.onSparkJobStarted(this, i); + } + } + } + } + public void setSparkCounters(SparkCounters sparkCounters) { this.sparkCounters = sparkCounters; } @SuppressWarnings("unchecked") void setSuccess(Object result) { - promise.setSuccess((T) result); + // The synchronization here is not necessary, but tests depend on it. + synchronized (listeners) { + promise.setSuccess((T) result); + changeState(State.SUCCEEDED); + } } void setFailure(Throwable error) { - promise.setFailure(error); + // The synchronization here is not necessary, but tests depend on it. + synchronized (listeners) { + promise.setFailure(error); + changeState(State.FAILED); + } } - /** Last attempt resort at preventing stray jobs from accumulating in SparkClientImpl. */ + /** + * Changes the state of this job handle, making sure that illegal state transitions are ignored. + * Fires events appropriately. + * + * As a rule, state transitions can only occur if the current state is "higher" than the current + * state (i.e., has a higher ordinal number) and is not a "final" state. "Final" states are + * CANCELLED, FAILED and SUCCEEDED, defined here in the code as having an ordinal number higher + * than the CANCELLED enum constant. + */ + boolean changeState(State newState) { + synchronized (listeners) { + if (newState.ordinal() > state.ordinal() && state.ordinal() < State.CANCELLED.ordinal()) { + state = newState; + for (Listener l : listeners) { + fireStateChange(newState, l); + } + return true; + } + return false; + } + } + + void addSparkJobId(int sparkJobId) { + synchronized (listeners) { + sparkJobIds.add(sparkJobId); + for (Listener l : listeners) { + l.onSparkJobStarted(this, sparkJobId); + } + } + } + + private void fireStateChange(State s, Listener l) { + switch (s) { + case SENT: + break; + case QUEUED: + l.onJobQueued(this); + break; + case STARTED: + l.onJobStarted(this); + break; + case CANCELLED: + l.onJobCancelled(this); + break; + case FAILED: + l.onJobFailed(this, promise.cause()); + break; + case SUCCEEDED: + try { + l.onJobSucceeded(this, promise.get()); + } catch (Exception e) { + // Shouldn't really happen. + throw new IllegalStateException(e); + } + break; + default: + throw new IllegalStateException(); + } + } + + /** Last attempt at preventing stray jobs from accumulating in SparkClientImpl. */ @Override protected void finalize() { if (!isDone()) { Index: spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java (working copy) @@ -77,9 +77,9 @@ private static final Logger LOG = LoggerFactory.getLogger(RemoteDriver.class); private final Map> activeJobs; + private final Object jcLock; private final Object shutdownLock; private final ExecutorService executor; - private final JobContextImpl jc; private final NioEventLoopGroup egroup; private final Rpc clientRpc; private final DriverProtocol protocol; @@ -87,10 +87,14 @@ // Used to queue up requests while the SparkContext is being created. private final List> jobQueue = Lists.newLinkedList(); - private boolean running; + // jc is effectively final, but it has to be volatile since it's accessed by different + // threads while the constructor is running. + private volatile JobContextImpl jc; + private volatile boolean running; private RemoteDriver(String[] args) throws Exception { this.activeJobs = Maps.newConcurrentMap(); + this.jcLock = new Object(); this.shutdownLock = new Object(); SparkConf conf = new SparkConf(); @@ -120,6 +124,7 @@ Map mapConf = Maps.newHashMap(); for (Tuple2 e : conf.getAll()) { mapConf.put(e._1(), e._2()); + LOG.debug("Remote Driver configured with: " + e._1() + "=" + e._2()); } String secret = mapConf.get(SparkClientFactory.CONF_KEY_SECRET); @@ -150,14 +155,20 @@ try { JavaSparkContext sc = new JavaSparkContext(conf); sc.sc().addSparkListener(new ClientListener()); - jc = new JobContextImpl(sc); + synchronized (jcLock) { + jc = new JobContextImpl(sc); + jcLock.notifyAll(); + } } catch (Exception e) { LOG.error("Failed to start SparkContext.", e); shutdown(e); + synchronized (jcLock) { + jcLock.notifyAll(); + } throw e; } - synchronized (jobQueue) { + synchronized (jcLock) { for (Iterator> it = jobQueue.iterator(); it.hasNext();) { it.next().submit(); } @@ -174,7 +185,7 @@ } private void submit(JobWrapper job) { - synchronized (jobQueue) { + synchronized (jcLock) { if (jc != null) { job.submit(); } else { @@ -235,6 +246,10 @@ clientRpc.call(new JobResult(jobId, result, error, counters)); } + void jobStarted(String jobId) { + clientRpc.call(new JobStarted(jobId)); + } + void jobSubmitted(String jobId, int sparkJobId) { LOG.debug("Send job({}/{}) submitted to Client.", jobId, sparkJobId); clientRpc.call(new JobSubmitted(jobId, sparkJobId)); @@ -264,6 +279,35 @@ submit(wrapper); } + private Object handle(ChannelHandlerContext ctx, SyncJobRequest msg) throws Exception { + // In case the job context is not up yet, let's wait, since this is supposed to be a + // "synchronous" RPC. + if (jc == null) { + synchronized (jcLock) { + while (jc == null) { + jcLock.wait(); + if (!running) { + throw new IllegalStateException("Remote context is shutting down."); + } + } + } + } + + jc.setMonitorCb(new MonitorCallback() { + @Override + public void call(JavaFutureAction future, + SparkCounters sparkCounters, Set cachedRDDIds) { + throw new IllegalStateException( + "JobContext.monitor() is not available for synchronous jobs."); + } + }); + try { + return msg.job.call(jc); + } finally { + jc.setMonitorCb(null); + } + } + } private class JobWrapper implements Callable { @@ -286,6 +330,8 @@ @Override public Void call() throws Exception { + protocol.jobStarted(req.id); + try { jc.setMonitorCb(new MonitorCallback() { @Override Index: spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java (working copy) @@ -38,6 +38,23 @@ JobHandle submit(Job job); /** + * Asks the remote context to run a job immediately. + *

+ * Normally, the remote context will queue jobs and execute them based on how many worker + * threads have been configured. This method will run the submitted job in the same thread + * processing the RPC message, so that queueing does not apply. + *

+ * It's recommended that this method only be used to run code that finishes quickly. This + * avoids interfering with the normal operation of the context. + *

+ * Note: the {@link JobContext#monitor()} functionality is not available when using this method. + * + * @param job The job to execute. + * @return A future to monitor the result of the job. + */ + Future run(Job job); + + /** * Stops the remote context. * * Any pending jobs will be cancelled, and the remote context will be torn down. Index: spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java (working copy) @@ -21,6 +21,7 @@ import java.util.Map; import org.apache.hadoop.hive.common.classification.InterfaceAudience; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hive.spark.client.rpc.RpcServer; import org.apache.spark.SparkException; @@ -67,12 +68,13 @@ /** * Instantiates a new Spark client. * - * @param conf Configuration for the remote Spark application. + * @param sparkConf Configuration for the remote Spark application, contains spark.* properties. + * @param hiveConf Configuration for Hive, contains hive.* properties. */ - public static synchronized SparkClient createClient(Map conf) + public static synchronized SparkClient createClient(Map sparkConf, HiveConf hiveConf) throws IOException, SparkException { Preconditions.checkState(server != null, "initialize() not called."); - return new SparkClientImpl(server, conf); + return new SparkClientImpl(server, sparkConf, hiveConf); } } Index: spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java (working copy) @@ -40,7 +40,9 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hive.spark.client.rpc.Rpc; +import org.apache.hive.spark.client.rpc.RpcConfiguration; import org.apache.hive.spark.client.rpc.RpcServer; import org.apache.spark.SparkContext; import org.apache.spark.SparkException; @@ -67,6 +69,7 @@ private static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath"; private final Map conf; + private final HiveConf hiveConf; private final AtomicInteger childIdGenerator; private final Thread driverThread; private final Map> jobs; @@ -74,8 +77,9 @@ private final ClientProtocol protocol; private volatile boolean isAlive; - SparkClientImpl(RpcServer rpcServer, Map conf) throws IOException, SparkException { + SparkClientImpl(RpcServer rpcServer, Map conf, HiveConf hiveConf) throws IOException, SparkException { this.conf = conf; + this.hiveConf = hiveConf; this.childIdGenerator = new AtomicInteger(); this.jobs = Maps.newConcurrentMap(); @@ -116,13 +120,16 @@ } @Override + public Future run(Job job) { + return protocol.run(job); + } + + @Override public void stop() { if (isAlive) { isAlive = false; try { - protocol.endSession().get(10, TimeUnit.SECONDS); - } catch (TimeoutException te) { - LOG.warn("Timed out waiting for driver to respond to stop request."); + protocol.endSession(); } catch (Exception e) { LOG.warn("Exception while waiting for end session reply.", e); } finally { @@ -137,29 +144,29 @@ LOG.debug("Interrupted before driver thread was finished."); } if (endTime - System.currentTimeMillis() <= 0) { - LOG.debug("Shut down time out."); + LOG.warn("Timed out shutting down remote driver, interrupting..."); driverThread.interrupt(); } } @Override public Future addJar(URL url) { - return submit(new AddJarJob(url.toString())); + return run(new AddJarJob(url.toString())); } @Override public Future addFile(URL url) { - return submit(new AddFileJob(url.toString())); + return run(new AddFileJob(url.toString())); } @Override public Future getExecutorCount() { - return submit(new GetExecutorCountJob()); + return run(new GetExecutorCountJob()); } @Override public Future getDefaultParallelism() { - return submit(new GetDefaultParallelismJob()); + return run(new GetDefaultParallelismJob()); } void cancel(String jobId) { @@ -296,7 +303,26 @@ argv.add("org.apache.spark.deploy.SparkSubmit"); } + if (master.equals("yarn-cluster")) { + String executorCores = conf.get("spark.executor.cores"); + if (executorCores != null) { + argv.add("--executor-cores"); + argv.add(executorCores); + } + String executorMemory = conf.get("spark.executor.memory"); + if (executorMemory != null) { + argv.add("--executor-memory"); + argv.add(executorMemory); + } + + String numOfExecutors = conf.get("spark.executor.instances"); + if (numOfExecutors != null) { + argv.add("--num-executors"); + argv.add(numOfExecutors); + } + } + argv.add("--properties-file"); argv.add(properties.getAbsolutePath()); argv.add("--class"); @@ -313,6 +339,14 @@ argv.add("--remote-port"); argv.add(serverPort); + //hive.spark.* keys are passed down to the RemoteDriver via --conf, + //as --properties-file contains the spark.* keys that are meant for SparkConf object. + for (String hiveSparkConfKey : RpcConfiguration.HIVE_SPARK_RSC_CONFIGS) { + String value = RpcConfiguration.getValue(hiveConf, hiveSparkConfKey); + argv.add("--conf"); + argv.add(String.format("%s=%s", hiveSparkConfKey, value)); + } + LOG.debug("Running client driver with argv: {}", Joiner.on(" ").join(argv)); ProcessBuilder pb = new ProcessBuilder(argv.toArray(new String[argv.size()])); @@ -360,7 +394,7 @@ JobHandleImpl submit(Job job) { final String jobId = UUID.randomUUID().toString(); final Promise promise = driverRpc.createPromise(); - JobHandleImpl handle = new JobHandleImpl(SparkClientImpl.this, promise, jobId); + final JobHandleImpl handle = new JobHandleImpl(SparkClientImpl.this, promise, jobId); jobs.put(jobId, handle); final io.netty.util.concurrent.Future rpc = driverRpc.call(new JobRequest(jobId, job)); @@ -371,7 +405,9 @@ rpc.addListener(new GenericFutureListener>() { @Override public void operationComplete(io.netty.util.concurrent.Future f) { - if (!f.isSuccess() && !promise.isDone()) { + if (f.isSuccess()) { + handle.changeState(JobHandle.State.QUEUED); + } else if (!promise.isDone()) { promise.setFailure(f.cause()); } } @@ -379,16 +415,24 @@ promise.addListener(new GenericFutureListener>() { @Override public void operationComplete(Promise p) { - jobs.remove(jobId); + if (jobId != null) { + jobs.remove(jobId); + } if (p.isCancelled() && !rpc.isDone()) { rpc.cancel(true); } } }); - return handle; } + Future run(Job job) { + @SuppressWarnings("unchecked") + final io.netty.util.concurrent.Future rpc = (io.netty.util.concurrent.Future) + driverRpc.call(new SyncJobRequest(job), Serializable.class); + return rpc; + } + void cancel(String jobId) { driverRpc.call(new CancelJob(jobId)); } @@ -426,11 +470,20 @@ } } + private void handle(ChannelHandlerContext ctx, JobStarted msg) { + JobHandleImpl handle = jobs.get(msg.id); + if (handle != null) { + handle.changeState(JobHandle.State.STARTED); + } else { + LOG.warn("Received event for unknown job {}", msg.id); + } + } + private void handle(ChannelHandlerContext ctx, JobSubmitted msg) { JobHandleImpl handle = jobs.get(msg.clientJobId); if (handle != null) { LOG.info("Received spark job ID: {} for {}", msg.sparkJobId, msg.clientJobId); - handle.getSparkJobIds().add(msg.sparkJobId); + handle.addSparkJobId(msg.sparkJobId); } else { LOG.warn("Received spark job ID: {} for unknown job {}", msg.sparkJobId, msg.clientJobId); } Index: spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java (working copy) @@ -84,7 +84,7 @@ final String secret, final RpcDispatcher dispatcher) throws Exception { final RpcConfiguration rpcConf = new RpcConfiguration(config); - int connectTimeoutMs = rpcConf.getConnectTimeoutMs(); + int connectTimeoutMs = (int) rpcConf.getConnectTimeoutMs(); final ChannelFuture cf = new Bootstrap() .group(eloop) Index: spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java (revision 1654728) +++ spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java (working copy) @@ -21,9 +21,14 @@ import java.net.Inet4Address; import java.net.InetAddress; import java.net.NetworkInterface; +import java.util.Arrays; import java.util.Enumeration; +import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; +import com.google.common.collect.ImmutableSet; +import org.apache.hadoop.hive.conf.HiveConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,63 +42,49 @@ private static final Logger LOG = LoggerFactory.getLogger(RpcConfiguration.class); - /** Connection timeout for RPC clients. */ - public static final String CONNECT_TIMEOUT_MS_KEY = "hive.spark.client.connect.timeout.ms"; - private static final int CONNECT_TIMEOUT_MS_DEFAULT = 1000; + public static final ImmutableSet HIVE_SPARK_RSC_CONFIGS = ImmutableSet.of( + HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT.varname, + HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT.varname, + HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname, + HiveConf.ConfVars.SPARK_RPC_MAX_MESSAGE_SIZE.varname, + HiveConf.ConfVars.SPARK_RPC_MAX_THREADS.varname, + HiveConf.ConfVars.SPARK_RPC_SECRET_RANDOM_BITS.varname + ); + public static final ImmutableSet HIVE_SPARK_TIME_CONFIGS = ImmutableSet.of( + HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT.varname, + HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT.varname + ); - /** - * How long the server should wait for clients to connect back after they're - * registered. Also used to time out the client waiting for the server to - * reply to its "hello" message. - */ - public static final String SERVER_CONNECT_TIMEOUT_MS_KEY = "hive.spark.client.server.connect.timeout.ms"; - private static final long SERVER_CONNECT_TIMEOUT_MS_DEFAULT = 10000L; - - /** - * Number of bits of randomness in the generated client secrets. Rounded down - * to the nearest multiple of 8. - */ - public static final String SECRET_RANDOM_BITS_KEY = "hive.spark.client.secret.bits"; - private static final int SECRET_RANDOM_BITS_DEFAULT = 256; - - /** Hostname or IP address to advertise for the server. */ public static final String SERVER_LISTEN_ADDRESS_KEY = "hive.spark.client.server.address"; - /** Maximum number of threads to use for the RPC event loop. */ - public static final String RPC_MAX_THREADS_KEY = "hive.spark.client.rpc.threads"; - public static final int RPC_MAX_THREADS_DEFAULT = 8; + private final Map config; - /** Maximum message size. Default = 10MB. */ - public static final String RPC_MAX_MESSAGE_SIZE_KEY = "hive.spark.client.rpc.max.size"; - public static final int RPC_MAX_MESSAGE_SIZE_DEFAULT = 50 * 1024 * 1024; + private static final HiveConf DEFAULT_CONF = new HiveConf(); - /** Channel logging level. */ - public static final String RPC_CHANNEL_LOG_LEVEL_KEY = "hive.spark.client.channel.log.level"; - - private final Map config; - public RpcConfiguration(Map config) { this.config = config; } - int getConnectTimeoutMs() { - String value = config.get(CONNECT_TIMEOUT_MS_KEY); - return value != null ? Integer.parseInt(value) : CONNECT_TIMEOUT_MS_DEFAULT; + long getConnectTimeoutMs() { + String value = config.get(HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT.varname); + return value != null ? Integer.parseInt(value) : DEFAULT_CONF.getTimeVar( + HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT, TimeUnit.MILLISECONDS); } int getMaxMessageSize() { - String value = config.get(RPC_MAX_MESSAGE_SIZE_KEY); - return value != null ? Integer.parseInt(value) : RPC_MAX_MESSAGE_SIZE_DEFAULT; + String value = config.get(HiveConf.ConfVars.SPARK_RPC_MAX_MESSAGE_SIZE.varname); + return value != null ? Integer.parseInt(value) : HiveConf.ConfVars.SPARK_RPC_MAX_MESSAGE_SIZE.defaultIntVal; } long getServerConnectTimeoutMs() { - String value = config.get(SERVER_CONNECT_TIMEOUT_MS_KEY); - return value != null ? Long.parseLong(value) : SERVER_CONNECT_TIMEOUT_MS_DEFAULT; + String value = config.get(HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT.varname); + return value != null ? Long.parseLong(value) : DEFAULT_CONF.getTimeVar( + HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT, TimeUnit.MILLISECONDS); } int getSecretBits() { - String value = config.get(SECRET_RANDOM_BITS_KEY); - return value != null ? Integer.parseInt(value) : SECRET_RANDOM_BITS_DEFAULT; + String value = config.get(HiveConf.ConfVars.SPARK_RPC_SECRET_RANDOM_BITS.varname); + return value != null ? Integer.parseInt(value) : HiveConf.ConfVars.SPARK_RPC_SECRET_RANDOM_BITS.defaultIntVal; } String getServerAddress() throws IOException { @@ -133,12 +124,28 @@ } String getRpcChannelLogLevel() { - return config.get(RPC_CHANNEL_LOG_LEVEL_KEY); + return config.get(HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname); } public int getRpcThreadCount() { - String value = config.get(RPC_MAX_THREADS_KEY); - return value != null ? Integer.parseInt(value) : RPC_MAX_THREADS_DEFAULT; + String value = config.get(HiveConf.ConfVars.SPARK_RPC_MAX_THREADS.varname); + return value != null ? Integer.parseInt(value) : HiveConf.ConfVars.SPARK_RPC_MAX_THREADS.defaultIntVal; } + + /** + * Utility method for a given RpcConfiguration key, to convert value to millisecond if it is a time value, + * and return as string in either case. + * @param conf hive configuration + * @param key Rpc configuration to lookup (hive.spark.*) + * @return string form of the value + */ + public static String getValue(HiveConf conf, String key) { + if (HIVE_SPARK_TIME_CONFIGS.contains(key)) { + HiveConf.ConfVars confVar = HiveConf.getConfVars(key); + return String.valueOf(conf.getTimeVar(confVar, TimeUnit.MILLISECONDS)); + } else { + return conf.get(key); + } + } } Index: spark-client/src/test/java/org/apache/hive/spark/client/TestJobHandle.java =================================================================== --- spark-client/src/test/java/org/apache/hive/spark/client/TestJobHandle.java (revision 0) +++ spark-client/src/test/java/org/apache/hive/spark/client/TestJobHandle.java (working copy) @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.spark.client; + +import java.io.Serializable; + +import io.netty.util.concurrent.Promise; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.InOrder; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +@RunWith(MockitoJUnitRunner.class) +public class TestJobHandle { + + @Mock private SparkClientImpl client; + @Mock private Promise promise; + @Mock private JobHandle.Listener listener; + @Mock private JobHandle.Listener listener2; + + @Test + public void testStateChanges() throws Exception { + JobHandleImpl handle = new JobHandleImpl(client, promise, "job"); + handle.addListener(listener); + + assertTrue(handle.changeState(JobHandle.State.QUEUED)); + verify(listener).onJobQueued(handle); + + assertTrue(handle.changeState(JobHandle.State.STARTED)); + verify(listener).onJobStarted(handle); + + handle.addSparkJobId(1); + verify(listener).onSparkJobStarted(same(handle), eq(1)); + + assertTrue(handle.changeState(JobHandle.State.CANCELLED)); + verify(listener).onJobCancelled(handle); + + assertFalse(handle.changeState(JobHandle.State.STARTED)); + assertFalse(handle.changeState(JobHandle.State.FAILED)); + assertFalse(handle.changeState(JobHandle.State.SUCCEEDED)); + } + + @Test + public void testFailedJob() throws Exception { + JobHandleImpl handle = new JobHandleImpl(client, promise, "job"); + handle.addListener(listener); + + Throwable cause = new Exception(); + when(promise.cause()).thenReturn(cause); + + assertTrue(handle.changeState(JobHandle.State.FAILED)); + verify(promise).cause(); + verify(listener).onJobFailed(handle, cause); + } + + @Test + public void testSucceededJob() throws Exception { + JobHandleImpl handle = new JobHandleImpl(client, promise, "job"); + handle.addListener(listener); + + Serializable result = new Exception(); + when(promise.get()).thenReturn(result); + + assertTrue(handle.changeState(JobHandle.State.SUCCEEDED)); + verify(promise).get(); + verify(listener).onJobSucceeded(handle, result); + } + + @Test + public void testImmediateCallback() throws Exception { + JobHandleImpl handle = new JobHandleImpl(client, promise, "job"); + assertTrue(handle.changeState(JobHandle.State.QUEUED)); + handle.addListener(listener); + verify(listener).onJobQueued(handle); + + handle.changeState(JobHandle.State.STARTED); + handle.addSparkJobId(1); + handle.changeState(JobHandle.State.CANCELLED); + + handle.addListener(listener2); + InOrder inOrder = inOrder(listener2); + inOrder.verify(listener2).onSparkJobStarted(same(handle), eq(1)); + inOrder.verify(listener2).onJobCancelled(same(handle)); + } + +} Index: spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java =================================================================== --- spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java (revision 1654728) +++ spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java (working copy) @@ -17,40 +17,41 @@ package org.apache.hive.spark.client; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.InputStream; +import java.io.Serializable; import java.net.URL; import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.jar.JarOutputStream; import java.util.zip.ZipEntry; +import com.google.common.base.Objects; +import com.google.common.base.Strings; +import com.google.common.io.ByteStreams; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hive.spark.counter.SparkCounters; +import org.apache.spark.SparkException; import org.apache.spark.SparkFiles; import org.apache.spark.api.java.JavaFutureAction; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.VoidFunction; import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; -import com.google.common.base.Objects; -import com.google.common.base.Strings; -import com.google.common.io.ByteStreams; - public class TestSparkClient { // Timeouts are bad... mmmkay. private static final long TIMEOUT = 20; + private static final HiveConf HIVECONF = new HiveConf(); private Map createConf(boolean local) { Map conf = new HashMap(); @@ -78,8 +79,19 @@ runTest(true, new TestFunction() { @Override public void call(SparkClient client) throws Exception { + JobHandle.Listener listener = newListener(); JobHandle handle = client.submit(new SimpleJob()); + handle.addListener(listener); assertEquals("hello", handle.get(TIMEOUT, TimeUnit.SECONDS)); + + // Try an invalid state transition on the handle. This ensures that the actual state + // change we're interested in actually happened, since internally the handle serializes + // state changes. + assertFalse(((JobHandleImpl)handle).changeState(JobHandle.State.SENT)); + + verify(listener).onJobQueued(handle); + verify(listener).onJobStarted(handle); + verify(listener).onJobSucceeded(same(handle), eq(handle.get())); } }); } @@ -100,17 +112,41 @@ runTest(true, new TestFunction() { @Override public void call(SparkClient client) throws Exception { - JobHandle handle = client.submit(new SimpleJob()); + JobHandle.Listener listener = newListener(); + JobHandle handle = client.submit(new ErrorJob()); + handle.addListener(listener); try { handle.get(TIMEOUT, TimeUnit.SECONDS); + fail("Should have thrown an exception."); } catch (ExecutionException ee) { - assertTrue(ee.getCause() instanceof IllegalStateException); + assertTrue(ee.getCause() instanceof SparkException); + assertTrue(ee.getCause().getMessage().contains("IllegalStateException: Hello")); } + + // Try an invalid state transition on the handle. This ensures that the actual state + // change we're interested in actually happened, since internally the handle serializes + // state changes. + assertFalse(((JobHandleImpl)handle).changeState(JobHandle.State.SENT)); + + verify(listener).onJobQueued(handle); + verify(listener).onJobStarted(handle); + verify(listener).onJobFailed(same(handle), any(Throwable.class)); } }); } @Test + public void testSyncRpc() throws Exception { + runTest(true, new TestFunction() { + @Override + public void call(SparkClient client) throws Exception { + Future result = client.run(new SyncRpc()); + assertEquals("Hello", result.get(TIMEOUT, TimeUnit.SECONDS)); + } + }); + } + + @Test public void testRemoteClient() throws Exception { runTest(false, new TestFunction() { @Override @@ -126,18 +162,26 @@ runTest(true, new TestFunction() { @Override public void call(SparkClient client) throws Exception { + JobHandle.Listener listener = newListener(); JobHandle future = client.submit(new AsyncSparkJob()); + future.addListener(listener); future.get(TIMEOUT, TimeUnit.SECONDS); MetricsCollection metrics = future.getMetrics(); assertEquals(1, metrics.getJobIds().size()); assertTrue(metrics.getAllMetrics().executorRunTime > 0L); + verify(listener).onSparkJobStarted(same(future), + eq(metrics.getJobIds().iterator().next())); + JobHandle.Listener listener2 = newListener(); JobHandle future2 = client.submit(new AsyncSparkJob()); + future2.addListener(listener2); future2.get(TIMEOUT, TimeUnit.SECONDS); MetricsCollection metrics2 = future2.getMetrics(); assertEquals(1, metrics2.getJobIds().size()); assertFalse(Objects.equal(metrics.getJobIds(), metrics2.getJobIds())); assertTrue(metrics2.getAllMetrics().executorRunTime > 0L); + verify(listener2).onSparkJobStarted(same(future2), + eq(metrics2.getJobIds().iterator().next())); } }); } @@ -214,13 +258,20 @@ }); } + private JobHandle.Listener newListener() { + @SuppressWarnings("unchecked") + JobHandle.Listener listener = + (JobHandle.Listener) mock(JobHandle.Listener.class); + return listener; + } + private void runTest(boolean local, TestFunction test) throws Exception { Map conf = createConf(local); SparkClientFactory.initialize(conf); SparkClient client = null; try { test.config(conf); - client = SparkClientFactory.createClient(conf); + client = SparkClientFactory.createClient(conf, HIVECONF); test.call(client); } finally { if (client != null) { @@ -239,6 +290,15 @@ } + private static class ErrorJob implements Job { + + @Override + public String call(JobContext jc) { + throw new IllegalStateException("Hello"); + } + + } + private static class SparkJob implements Job { @Override @@ -333,6 +393,15 @@ } + private static class SyncRpc implements Job { + + @Override + public String call(JobContext jc) { + return "Hello"; + } + + } + private abstract static class TestFunction { abstract void call(SparkClient client) throws Exception; void config(Map conf) { } Index: spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java =================================================================== --- spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java (revision 1654728) +++ spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java (working copy) @@ -32,6 +32,7 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.concurrent.Future; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hive.conf.HiveConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +50,7 @@ private Collection closeables; private Map emptyConfig = - ImmutableMap.of(RpcConfiguration.RPC_CHANNEL_LOG_LEVEL_KEY, "DEBUG"); + ImmutableMap.of(HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname, "DEBUG"); @Before public void setUp() {