From 8986aa9b1cde7e116874eb4b9bf5cf6142a244a4 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 8 Sep 2016 03:25:51 +0000 Subject: [PATCH 01/50] Stop Spark Application if launcher goes down and use reflection --- .../spark/launcher/LauncherBackend.scala | 49 ++++++- .../launcher/AbstractSparkAppHandle.java | 138 ++++++++++++++++++ .../spark/launcher/ChildProcAppHandle.java | 110 +------------- .../spark/launcher/ChildThreadAppHandle.java | 54 +++++++ .../spark/launcher/LauncherConnection.java | 1 - .../spark/launcher/LauncherProtocol.java | 3 + .../apache/spark/launcher/LauncherServer.java | 47 ++++-- .../apache/spark/launcher/SparkLauncher.java | 102 ++++++++++--- .../spark/launcher/SparkSubmitRunner.java | 59 ++++++++ .../org/apache/spark/deploy/yarn/Client.scala | 10 +- 10 files changed, 434 insertions(+), 139 deletions(-) create mode 100644 launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala index a5d41a1eeb47..d6f30ad18254 100644 --- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala +++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala @@ -17,9 +17,11 @@ package org.apache.spark.launcher +import java.io.IOException import java.net.{InetAddress, Socket} import org.apache.spark.SPARK_VERSION +import org.apache.spark.internal.Logging import org.apache.spark.launcher.LauncherProtocol._ import org.apache.spark.util.{ThreadUtils, Utils} @@ -29,26 +31,63 @@ import org.apache.spark.util.{ThreadUtils, Utils} * * See `LauncherServer` for an explanation of how launcher communication works. */ -private[spark] abstract class LauncherBackend { +private[spark] abstract class LauncherBackend extends Logging { private var clientThread: Thread = _ private var connection: BackendConnection = _ private var lastState: SparkAppHandle.State = _ + private var stopFlag: Boolean = false @volatile private var _isConnected = false def connect(): Unit = { val port = sys.env.get(LauncherProtocol.ENV_LAUNCHER_PORT).map(_.toInt) val secret = sys.env.get(LauncherProtocol.ENV_LAUNCHER_SECRET) + val stopFlag = sys.env.get(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG).map(_.toBoolean) if (port != None && secret != None) { - val s = new Socket(InetAddress.getLoopbackAddress(), port.get) + if(stopFlag != None) { + connect(port.get, secret.get, stopFlag.get) + } else { + connect(port.get, secret.get) + } + } + } + + def connect(port: Int, secret: String): Unit = { + if (port != None && secret != None) { + val s = new Socket(InetAddress.getLoopbackAddress(), port) connection = new BackendConnection(s) - connection.send(new Hello(secret.get, SPARK_VERSION)) + connection.send(new Hello(secret, SPARK_VERSION)) clientThread = LauncherBackend.threadFactory.newThread(connection) clientThread.start() _isConnected = true + if(stopFlag) { + val shutdownHook: Runnable = new Runnable() { + def run { + logInfo("LauncherBackend shutdown hook invoked..") + try { + if(_isConnected && stopFlag) { + onStopRequest() + } + } + catch { + case anotherIOE: IOException => { + logInfo("Error while running LauncherBackend shutdownHook...", anotherIOE) + } + } + } + } + + val shutdownHookThread: Thread = LauncherBackend.threadFactory.newThread(shutdownHook) + Runtime.getRuntime.addShutdownHook(shutdownHookThread) + } } } + def connect(port: Int, secret: String, stopFlag: Boolean): Unit = { + this.stopFlag = stopFlag + connect(port, secret) + } + def close(): Unit = { if (connection != null) { try { @@ -71,6 +110,9 @@ private[spark] abstract class LauncherBackend { if (connection != null && lastState != state) { connection.send(new SetState(state)) lastState = state + if(!_isConnected && stopFlag) { + fireStopRequest() + } } } @@ -115,7 +157,6 @@ private[spark] abstract class LauncherBackend { _isConnected = false } } - } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java new file mode 100644 index 000000000000..ee1188f2e022 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java @@ -0,0 +1,138 @@ +/* + * 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.spark.launcher; + +import java.io.IOException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.logging.Level; +import java.util.logging.Logger; + + +public abstract class AbstractSparkAppHandle implements SparkAppHandle { + private static final Logger LOG = Logger.getLogger(AbstractSparkAppHandle.class.getName()); + protected final String secret; + protected final LauncherServer server; + protected boolean disposed; + protected List listeners; + protected State state; + private LauncherConnection connection; + private String appId; + + OutputRedirector redirector; + + + + public AbstractSparkAppHandle(LauncherServer server, String secret) { + this.server = server; + this.secret = secret; + this.state = State.UNKNOWN; + } + + @Override + public synchronized void addListener(Listener l) { + if (listeners == null) { + listeners = new ArrayList<>(); + } + listeners.add(l); + } + + @Override + public State getState() { + return state; + } + + @Override + public String getAppId() { + return appId; + } + + @Override + public void stop() { + CommandBuilderUtils.checkState(connection != null, "Application is still not connected."); + try { + connection.send(new LauncherProtocol.Stop()); + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + } + + @Override + public synchronized void disconnect() { + if (!disposed) { + disposed = true; + if (connection != null) { + try { + connection.close(); + } catch (IOException ioe) { + // no-op. + } + } + server.unregister(this); + if (redirector != null) { + redirector.stop(); + } + } + } + + String getSecret() { + return secret; + } + + void setConnection(LauncherConnection connection) { + this.connection = connection; + } + + LauncherServer getServer() { + return server; + } + + LauncherConnection getConnection() { + return connection; + } + + void setState(State s) { + if (!state.isFinal()) { + state = s; + fireEvent(false); + } else { + LOG.log(Level.WARNING, "Backend requested transition from final state {0} to {1}.", + new Object[]{state, s}); + } + } + + void setAppId(String appId) { + this.appId = appId; + fireEvent(true); + } + + private synchronized void fireEvent(boolean isInfoChanged) { + if (listeners != null) { + for (Listener l : listeners) { + if (isInfoChanged) { + l.infoChanged(this); + } else { + l.stateChanged(this); + } + } + } + } + +} + diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java index c0779e1c4e9a..f7bcc1c1b25c 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java @@ -27,71 +27,14 @@ /** * Handle implementation for monitoring apps started as a child process. */ -class ChildProcAppHandle implements SparkAppHandle { +class ChildProcAppHandle extends AbstractSparkAppHandle { private static final Logger LOG = Logger.getLogger(ChildProcAppHandle.class.getName()); - private final String secret; - private final LauncherServer server; - private Process childProc; - private boolean disposed; - private LauncherConnection connection; - private List listeners; - private State state; - private String appId; - private OutputRedirector redirector; ChildProcAppHandle(String secret, LauncherServer server) { - this.secret = secret; - this.server = server; - this.state = State.UNKNOWN; - } - - @Override - public synchronized void addListener(Listener l) { - if (listeners == null) { - listeners = new ArrayList<>(); - } - listeners.add(l); - } - - @Override - public State getState() { - return state; - } - - @Override - public String getAppId() { - return appId; - } - - @Override - public void stop() { - CommandBuilderUtils.checkState(connection != null, "Application is still not connected."); - try { - connection.send(new LauncherProtocol.Stop()); - } catch (IOException ioe) { - throw new RuntimeException(ioe); - } - } - - @Override - public synchronized void disconnect() { - if (!disposed) { - disposed = true; - if (connection != null) { - try { - connection.close(); - } catch (IOException ioe) { - // no-op. - } - } - server.unregister(this); - if (redirector != null) { - redirector.stop(); - } - } + super(server, secret); } @Override @@ -103,13 +46,10 @@ public synchronized void kill() { try { childProc.exitValue(); } catch (IllegalThreadStateException e) { - // Child is still alive. Try to use Java 8's "destroyForcibly()" if available, - // fall back to the old API if it's not there. try { - Method destroy = childProc.getClass().getMethod("destroyForcibly"); - destroy.invoke(childProc); - } catch (Exception inner) { childProc.destroy(); + } catch (Exception inner) { + // no-op } } finally { childProc = null; @@ -117,53 +57,11 @@ public synchronized void kill() { } } - String getSecret() { - return secret; - } - void setChildProc(Process childProc, String loggerName) { this.childProc = childProc; this.redirector = new OutputRedirector(childProc.getInputStream(), loggerName, SparkLauncher.REDIRECTOR_FACTORY); } - void setConnection(LauncherConnection connection) { - this.connection = connection; - } - - LauncherServer getServer() { - return server; - } - - LauncherConnection getConnection() { - return connection; - } - - void setState(State s) { - if (!state.isFinal()) { - state = s; - fireEvent(false); - } else { - LOG.log(Level.WARNING, "Backend requested transition from final state {0} to {1}.", - new Object[] { state, s }); - } - } - - void setAppId(String appId) { - this.appId = appId; - fireEvent(true); - } - - private synchronized void fireEvent(boolean isInfoChanged) { - if (listeners != null) { - for (Listener l : listeners) { - if (isInfoChanged) { - l.infoChanged(this); - } else { - l.stateChanged(this); - } - } - } - } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java new file mode 100644 index 000000000000..c2b5eb13cef5 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java @@ -0,0 +1,54 @@ +/* + * 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.spark.launcher; + +import java.util.logging.Logger; + +public class ChildThreadAppHandle extends AbstractSparkAppHandle { + private static final Logger LOG = Logger.getLogger(ChildThreadAppHandle.class.getName()); + private Thread childThread; + + public ChildThreadAppHandle(String secret, LauncherServer server) { + super(server, secret); + } + + @Override + public synchronized void kill() { + if (!disposed) { + disconnect(); + } + if (childThread!= null) { + try { + childThread.join(3000); + } catch (IllegalThreadStateException | InterruptedException e) { + try { + childThread.interrupt(); + } catch (Exception inner) { + LOG.info("Failed to stop Thread: "+ inner.getMessage()); + } + } finally { + childThread = null; + } + } + + } + + void setChildThread(Thread childThread) { + this.childThread = childThread; + } +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java index eec264909bbb..5991ec80a2bd 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java @@ -106,5 +106,4 @@ public void close() throws IOException { } } } - } diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java index 042f11cd9e43..5081ac213812 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java @@ -32,6 +32,9 @@ final class LauncherProtocol { /** Environment variable where the secret for connecting back to the server is stored. */ static final String ENV_LAUNCHER_SECRET = "_SPARK_LAUNCHER_SECRET"; + /** Environment variable where the stop if Launcher is stored. */ + static final String ENV_LAUNCHER_STOP_FLAG = "_SPARK_LAUNCHER_STOP_FLAG"; + static class Message implements Serializable { } diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index ae43f563e8b4..0dbf3d0497e2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -89,11 +89,32 @@ class LauncherServer implements Closeable { private static volatile LauncherServer serverInstance; /** - * Creates a handle for an app to be launched. This method will start a server if one hasn't been + * Creates a handle for an app to be launched using ChildProc. + * This method will start a server if one hasn't been * started yet. The server is shared for multiple handles, and once all handles are disposed of, * the server is shut down. */ static synchronized ChildProcAppHandle newAppHandle() throws IOException { + return (ChildProcAppHandle) LauncherServer.newAppHandle(false); + } + + /** + * Creates a handle for an app to be launched from a Thread within the current JVM. + * This method will start a server if one hasn't been + * started yet. The server is shared for multiple handles, and once all handles are disposed of, + * the server is shut down. + */ + static synchronized ChildThreadAppHandle newAppThreadHandle() throws IOException { + return (ChildThreadAppHandle) LauncherServer.newAppHandle(true); + } + + + /** + * Creates a handle for an app to be launched. This method will start a server if one hasn't been + * started yet. The server is shared for multiple handles, and once all handles are disposed of, + * the server is shut down. + */ + static synchronized AbstractSparkAppHandle newAppHandle(boolean isThreadHandle) throws IOException { LauncherServer server = serverInstance != null ? serverInstance : new LauncherServer(); server.ref(); serverInstance = server; @@ -102,8 +123,7 @@ static synchronized ChildProcAppHandle newAppHandle() throws IOException { while (server.pending.containsKey(secret)) { secret = server.createSecret(); } - - return server.newAppHandle(secret); + return server.newAppHandle(secret, isThreadHandle); } static LauncherServer getServerInstance() { @@ -112,7 +132,7 @@ static LauncherServer getServerInstance() { private final AtomicLong refCount; private final AtomicLong threadIds; - private final ConcurrentMap pending; + private final ConcurrentMap pending; private final List clients; private final ServerSocket server; private final Thread serverThread; @@ -157,9 +177,15 @@ public void run() { * Creates a new app handle. The handle will wait for an incoming connection for a configurable * amount of time, and if one doesn't arrive, it will transition to an error state. */ - ChildProcAppHandle newAppHandle(String secret) { - ChildProcAppHandle handle = new ChildProcAppHandle(secret, this); - ChildProcAppHandle existing = pending.putIfAbsent(secret, handle); + AbstractSparkAppHandle newAppHandle(String secret, boolean isThreadHandle) { + + AbstractSparkAppHandle handle; + if(isThreadHandle) { + handle = new ChildThreadAppHandle(secret, this); + } else { + handle = new ChildProcAppHandle(secret, this); + } + AbstractSparkAppHandle existing = pending.putIfAbsent(secret, handle); CommandBuilderUtils.checkState(existing == null, "Multiple handles with the same secret."); return handle; } @@ -180,6 +206,7 @@ public void close() throws IOException { } } } + if (serverThread != null) { try { serverThread.join(); @@ -215,7 +242,7 @@ int getPort() { * Removes the client handle from the pending list (in case it's still there), and unrefs * the server. */ - void unregister(ChildProcAppHandle handle) { + void unregister(AbstractSparkAppHandle handle) { pending.remove(handle.getSecret()); unref(); } @@ -282,7 +309,7 @@ private String createSecret() { private class ServerConnection extends LauncherConnection { private TimerTask timeout; - private ChildProcAppHandle handle; + private AbstractSparkAppHandle handle; ServerConnection(Socket socket, TimerTask timeout) throws IOException { super(socket); @@ -296,7 +323,7 @@ protected void handle(Message msg) throws IOException { timeout.cancel(); timeout = null; Hello hello = (Hello) msg; - ChildProcAppHandle handle = pending.remove(hello.secret); + AbstractSparkAppHandle handle = pending.remove(hello.secret); if (handle != null) { handle.setConnection(this); handle.setState(SparkAppHandle.State.CONNECTED); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index ea56214d2390..baf81f793976 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; +import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -73,6 +74,21 @@ public class SparkLauncher { /** Logger name to use when launching a child process. */ public static final String CHILD_PROCESS_LOGGER_NAME = "spark.launcher.childProcLoggerName"; + /** Launcher Server Port to use when launching a child process. */ + public static final String LAUNCHER_INTERNAL_PORT = "spark.launcher.internal.port"; + + /** Launcher Server sets this when launching a child process. */ + public static final String CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET = "spark.launcher.internal.secret"; + + /** Stop Flag if interrupted Launcher Server goes away. */ + public static final String CHILD_PROCESS_LAUNCHER_STOP_FLAG = "spark.launcher.internal.stop.flag"; + + public SparkLauncher stopIfInterrupted() { + this.stopIfInterrupted = true; + return this; + } + + private boolean stopIfInterrupted = false; /** * A special value for the resource that tells Spark to not try to process the app resource as a * file. This is useful when the class being executed is added to the application using other @@ -503,23 +519,7 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr // Only setup stderr + stdout to logger redirection if user has not otherwise configured output // redirection. if (loggerName == null) { - String appName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); - if (appName == null) { - if (builder.appName != null) { - appName = builder.appName; - } else if (builder.mainClass != null) { - int dot = builder.mainClass.lastIndexOf("."); - if (dot >= 0 && dot < builder.mainClass.length() - 1) { - appName = builder.mainClass.substring(dot + 1, builder.mainClass.length()); - } else { - appName = builder.mainClass; - } - } else if (builder.appResource != null) { - appName = new File(builder.appResource).getName(); - } else { - appName = String.valueOf(COUNTER.incrementAndGet()); - } - } + String appName = getAppName(); String loggerPrefix = getClass().getPackage().getName(); loggerName = String.format("%s.app.%s", loggerPrefix, appName); pb.redirectErrorStream(true); @@ -528,6 +528,7 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr pb.environment().put(LauncherProtocol.ENV_LAUNCHER_PORT, String.valueOf(LauncherServer.getServerInstance().getPort())); pb.environment().put(LauncherProtocol.ENV_LAUNCHER_SECRET, handle.getSecret()); + pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG, String.valueOf(stopIfInterrupted)); try { handle.setChildProc(pb.start(), loggerName); } catch (IOException ioe) { @@ -538,6 +539,71 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr return handle; } + private String getAppName() throws IOException { + String appName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); + if (appName == null) { + if (builder.appName != null) { + appName = builder.appName; + } else if (builder.mainClass != null) { + int dot = builder.mainClass.lastIndexOf("."); + if (dot >= 0 && dot < builder.mainClass.length() - 1) { + appName = builder.mainClass.substring(dot + 1, builder.mainClass.length()); + } else { + appName = builder.mainClass; + } + } else if (builder.appResource != null) { + appName = new File(builder.appResource).getName(); + } else { + appName = String.valueOf(COUNTER.incrementAndGet()); + } + } + return appName; + } + + /** + * Starts a Spark application. + *

+ * This method returns a handle that provides information about the running application and can + * be used to do basic interaction with it. + *

+ * The returned handle assumes that the application will instantiate a single SparkContext + * during its lifetime. Once that context reports a final state (one that indicates the + * SparkContext has stopped), the handle will not perform new state transitions, so anything + * that happens after that cannot be monitored. The underlying application is launched as + * a Thread, {@link SparkAppHandle#kill()} can still be used to kill the spark application. + *

+ * @since 2.1.0 + * @param listeners Listeners to add to the handle before the app is launched. + * @return A handle for the launched application. + */ + public SparkAppHandle startApplicationInProcess(SparkAppHandle.Listener... listeners) throws IOException { + + ChildThreadAppHandle handle = LauncherServer.newAppThreadHandle(); + for (SparkAppHandle.Listener l : listeners) { + handle.addListener(l); + } + + String appName = getAppName(); + setConf(LAUNCHER_INTERNAL_PORT,String.valueOf(LauncherServer.getServerInstance().getPort())); + setConf(CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET, handle.getSecret()); + setConf(CHILD_PROCESS_LAUNCHER_STOP_FLAG, String.valueOf(stopIfInterrupted)); + try { + //trying to see if method is available in the classpath. + Method main = SparkSubmitRunner.getSparkSubmitMain(); + Thread submitJobThread = new Thread(new SparkSubmitRunner(main, builder.buildSparkSubmitArgs())); + submitJobThread.setName(appName); + handle.setChildThread(submitJobThread); + submitJobThread.start(); + } catch (ClassNotFoundException cnfe) { + throw new IOException(cnfe); + } catch (NoSuchMethodException nsme) { + throw new IOException(nsme); + } + return handle; + + } + + private ProcessBuilder createBuilder() { List cmd = new ArrayList<>(); String script = isWindows() ? "spark-submit.cmd" : "spark-submit"; @@ -613,4 +679,6 @@ protected void handleExtraArgs(List extra) { } + + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java new file mode 100644 index 000000000000..eb421012bd5f --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java @@ -0,0 +1,59 @@ +/* + * 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.spark.launcher; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.List; + +class SparkSubmitRunner implements Runnable { + private Method main; + private final List args; + + SparkSubmitRunner(Method main, List args) { + this.main = main; + this.args = args; + } + /** + * Trying to see if method is available in the classpath. + */ + protected static Method getSparkSubmitMain() throws ClassNotFoundException, NoSuchMethodException { + Class cls = Class.forName("org.apache.spark.deploy.SparkSubmit"); + return cls.getDeclaredMethod("main", String[].class); + } + + @Override + public void run() { + try { + if(main == null) { + main = getSparkSubmitMain(); + } + Object argsObj = args.toArray(new String[args.size()]); + main.invoke(null, argsObj); + } catch (IllegalAccessException illAcEx) { + throw new RuntimeException(illAcEx); + } catch (InvocationTargetException invokEx) { + throw new RuntimeException(invokEx); + } catch (RuntimeException re) { + throw re; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + +} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 2398f0aea316..a16a7e77a7a2 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -73,6 +73,10 @@ private[spark] class Client( private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster" + private val launcherServerPort : Int = sparkConf.get("spark.launcher.internal.port", "0").toInt + private val launcherServerSecret : String = sparkConf.get("spark.launcher.internal.secret", "") + private val launcherServerStopFlag : Boolean = sparkConf.get("spark.launcher.internal.stop.flag", "false").toBoolean + // AM related configurations private val amMemory = if (isClusterMode) { sparkConf.get(DRIVER_MEMORY).toInt @@ -144,7 +148,11 @@ private[spark] class Client( def submitApplication(): ApplicationId = { var appId: ApplicationId = null try { - launcherBackend.connect() + if ( launcherServerSecret !=null && launcherServerSecret != "" && launcherServerPort != 0 && launcherServerStopFlag != null) { + launcherBackend.connect(launcherServerPort, launcherServerSecret, launcherServerStopFlag) + } else { + launcherBackend.connect() + } // Setup the credentials before doing anything else, // so we have don't have issues at any point. setupCredentials() From 99b1d1bf51f700504b23734b05d8bf91f2b5b79a Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 8 Sep 2016 16:43:26 +0000 Subject: [PATCH 02/50] Fixing coding style issues and review comments --- .../spark/launcher/LauncherBackend.scala | 30 ++++++++----------- .../org/apache/spark/deploy/yarn/Client.scala | 6 ++-- 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala index d6f30ad18254..fa789c9af916 100644 --- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala +++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala @@ -23,7 +23,7 @@ import java.net.{InetAddress, Socket} import org.apache.spark.SPARK_VERSION import org.apache.spark.internal.Logging import org.apache.spark.launcher.LauncherProtocol._ -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{ShutdownHookManager, ThreadUtils, Utils} /** * A class that can be used to talk to a launcher server. Users should extend this class to @@ -44,7 +44,7 @@ private[spark] abstract class LauncherBackend extends Logging { val secret = sys.env.get(LauncherProtocol.ENV_LAUNCHER_SECRET) val stopFlag = sys.env.get(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG).map(_.toBoolean) if (port != None && secret != None) { - if(stopFlag != None) { + if (stopFlag != None) { connect(port.get, secret.get, stopFlag.get) } else { connect(port.get, secret.get) @@ -61,24 +61,20 @@ private[spark] abstract class LauncherBackend extends Logging { clientThread.start() _isConnected = true if(stopFlag) { - val shutdownHook: Runnable = new Runnable() { - def run { - logInfo("LauncherBackend shutdown hook invoked..") - try { - if(_isConnected && stopFlag) { - onStopRequest() - } - } - catch { - case anotherIOE: IOException => { - logInfo("Error while running LauncherBackend shutdownHook...", anotherIOE) - } + logDebug("Adding shutdown hook") // force eager creation of logger + var _shutdownHookRef = ShutdownHookManager.addShutdownHook( + ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () => + logInfo("Invoking onStopRequest() from shutdown hook") + try { + if (_isConnected && stopFlag) { + onStopRequest() } } + catch { + case anotherIOE: IOException => + logError("Error while running LauncherBackend shutdownHook...", anotherIOE) + } } - - val shutdownHookThread: Thread = LauncherBackend.threadFactory.newThread(shutdownHook) - Runtime.getRuntime.addShutdownHook(shutdownHookThread) } } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a16a7e77a7a2..d9fea009eb68 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -75,7 +75,8 @@ private[spark] class Client( private val launcherServerPort : Int = sparkConf.get("spark.launcher.internal.port", "0").toInt private val launcherServerSecret : String = sparkConf.get("spark.launcher.internal.secret", "") - private val launcherServerStopFlag : Boolean = sparkConf.get("spark.launcher.internal.stop.flag", "false").toBoolean + private val launcherServerStopFlag : Boolean = sparkConf.get("spark.launcher.internal.stop.flag", + "false").toBoolean // AM related configurations private val amMemory = if (isClusterMode) { @@ -148,7 +149,8 @@ private[spark] class Client( def submitApplication(): ApplicationId = { var appId: ApplicationId = null try { - if ( launcherServerSecret !=null && launcherServerSecret != "" && launcherServerPort != 0 && launcherServerStopFlag != null) { + if ( launcherServerSecret !=null && launcherServerSecret != "" && launcherServerPort != 0 && + launcherServerStopFlag != null) { launcherBackend.connect(launcherServerPort, launcherServerSecret, launcherServerStopFlag) } else { launcherBackend.connect() From 70a67fb7a021a3149a4f93c85f490b2015a10050 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 8 Sep 2016 18:04:56 +0000 Subject: [PATCH 03/50] formatting fixes --- .../spark/launcher/LauncherBackend.scala | 43 ++++++++++--------- .../launcher/AbstractSparkAppHandle.java | 3 -- 2 files changed, 22 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala index fa789c9af916..a23625aa3931 100644 --- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala +++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala @@ -53,28 +53,26 @@ private[spark] abstract class LauncherBackend extends Logging { } def connect(port: Int, secret: String): Unit = { - if (port != None && secret != None) { - val s = new Socket(InetAddress.getLoopbackAddress(), port) - connection = new BackendConnection(s) - connection.send(new Hello(secret, SPARK_VERSION)) - clientThread = LauncherBackend.threadFactory.newThread(connection) - clientThread.start() - _isConnected = true - if(stopFlag) { - logDebug("Adding shutdown hook") // force eager creation of logger - var _shutdownHookRef = ShutdownHookManager.addShutdownHook( - ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () => - logInfo("Invoking onStopRequest() from shutdown hook") - try { - if (_isConnected && stopFlag) { - onStopRequest() - } - } - catch { - case anotherIOE: IOException => - logError("Error while running LauncherBackend shutdownHook...", anotherIOE) + val s = new Socket(InetAddress.getLoopbackAddress(), port) + connection = new BackendConnection(s) + connection.send(new Hello(secret, SPARK_VERSION)) + clientThread = LauncherBackend.threadFactory.newThread(connection) + clientThread.start() + _isConnected = true + if (stopFlag) { + logDebug("Adding shutdown hook") // force eager creation of logger + var _shutdownHookRef = ShutdownHookManager.addShutdownHook( + ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () => + logInfo("Invoking onStopRequest() from shutdown hook") + try { + if (_isConnected && stopFlag) { + onStopRequest() } } + catch { + case anotherIOE: IOException => + logError("Error while running LauncherBackend shutdownHook...", anotherIOE) + } } } } @@ -106,7 +104,7 @@ private[spark] abstract class LauncherBackend extends Logging { if (connection != null && lastState != state) { connection.send(new SetState(state)) lastState = state - if(!_isConnected && stopFlag) { + if (!_isConnected && stopFlag) { fireStopRequest() } } @@ -148,6 +146,9 @@ private[spark] abstract class LauncherBackend extends Logging { override def close(): Unit = { try { super.close() + if (!_isConnected && stopFlag) { + fireStopRequest() + } } finally { onDisconnected() _isConnected = false diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java index ee1188f2e022..a1e7a3b2353b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java @@ -37,8 +37,6 @@ public abstract class AbstractSparkAppHandle implements SparkAppHandle { OutputRedirector redirector; - - public AbstractSparkAppHandle(LauncherServer server, String secret) { this.server = server; this.secret = secret; @@ -135,4 +133,3 @@ private synchronized void fireEvent(boolean isInfoChanged) { } } - From bac5cd26c6b5778b449fb7665365b4773b93ae9e Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 8 Sep 2016 18:49:23 +0000 Subject: [PATCH 04/50] Remove extra empty lines --- .../java/org/apache/spark/launcher/AbstractSparkAppHandle.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java index a1e7a3b2353b..c1a83f67789f 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java @@ -34,7 +34,6 @@ public abstract class AbstractSparkAppHandle implements SparkAppHandle { protected State state; private LauncherConnection connection; private String appId; - OutputRedirector redirector; public AbstractSparkAppHandle(LauncherServer server, String secret) { @@ -131,5 +130,4 @@ private synchronized void fireEvent(boolean isInfoChanged) { } } } - } From 38a7e3a6060b548ccb3826f1002fcf1a546d5a16 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 9 Sep 2016 10:12:39 -0500 Subject: [PATCH 05/50] Formatting and code review comments --- .../apache/spark/launcher/ChildThreadAppHandle.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java index c2b5eb13cef5..c216fe1f4eae 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java @@ -28,27 +28,26 @@ public ChildThreadAppHandle(String secret, LauncherServer server) { } @Override - public synchronized void kill() { + public synchronized void kill() { if (!disposed) { disconnect(); } - if (childThread!= null) { + if (childThread != null) { try { childThread.join(3000); - } catch (IllegalThreadStateException | InterruptedException e) { + } catch (InterruptedException e) { try { childThread.interrupt(); } catch (Exception inner) { - LOG.info("Failed to stop Thread: "+ inner.getMessage()); + LOG.info("Failed to stop Thread: " + inner.getMessage()); } } finally { childThread = null; } } - } void setChildThread(Thread childThread) { this.childThread = childThread; } -} +} \ No newline at end of file From 67f8de74ae59016c546ac4bf81bc130e3066a1f4 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 9 Sep 2016 14:42:35 -0500 Subject: [PATCH 06/50] Remove unwanted conditional to null check Boolean value --- yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index d9fea009eb68..2ab36aea38b8 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -149,8 +149,7 @@ private[spark] class Client( def submitApplication(): ApplicationId = { var appId: ApplicationId = null try { - if ( launcherServerSecret !=null && launcherServerSecret != "" && launcherServerPort != 0 && - launcherServerStopFlag != null) { + if ( launcherServerSecret !=null && launcherServerSecret != "" && launcherServerPort != 0) { launcherBackend.connect(launcherServerPort, launcherServerSecret, launcherServerStopFlag) } else { launcherBackend.connect() From 57defa950532c04420c6ae39ac680cdfa1bcaf13 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 14 Sep 2016 14:24:26 -0500 Subject: [PATCH 07/50] Addressing code review comments --- .../spark/launcher/LauncherBackend.scala | 2 +- .../apache/spark/launcher/LauncherServer.java | 2 +- .../apache/spark/launcher/SparkLauncher.java | 126 ++++++++++-------- .../spark/launcher/SparkSubmitRunner.java | 30 +++-- .../org/apache/spark/deploy/yarn/Client.scala | 2 +- 5 files changed, 92 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala index a23625aa3931..825750b650cc 100644 --- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala +++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala @@ -146,7 +146,7 @@ private[spark] abstract class LauncherBackend extends Logging { override def close(): Unit = { try { super.close() - if (!_isConnected && stopFlag) { + if (stopFlag) { fireStopRequest() } } finally { diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index 0dbf3d0497e2..2d5c9a9ee4e8 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -180,7 +180,7 @@ public void run() { AbstractSparkAppHandle newAppHandle(String secret, boolean isThreadHandle) { AbstractSparkAppHandle handle; - if(isThreadHandle) { + if (isThreadHandle) { handle = new ChildThreadAppHandle(secret, this); } else { handle = new ChildProcAppHandle(secret, this); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index baf81f793976..574c1dc03b5a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -74,21 +74,12 @@ public class SparkLauncher { /** Logger name to use when launching a child process. */ public static final String CHILD_PROCESS_LOGGER_NAME = "spark.launcher.childProcLoggerName"; - /** Launcher Server Port to use when launching a child process. */ - public static final String LAUNCHER_INTERNAL_PORT = "spark.launcher.internal.port"; + private static final String LAUNCHER_INTERNAL_PORT = "spark.launcher.internal.port"; - /** Launcher Server sets this when launching a child process. */ - public static final String CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET = "spark.launcher.internal.secret"; + private static final String CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET = "spark.launcher.internal.secret"; - /** Stop Flag if interrupted Launcher Server goes away. */ - public static final String CHILD_PROCESS_LAUNCHER_STOP_FLAG = "spark.launcher.internal.stop.flag"; + private static final String CHILD_PROCESS_LAUNCHER_STOP_FLAG = "spark.launcher.internal.stop.flag"; - public SparkLauncher stopIfInterrupted() { - this.stopIfInterrupted = true; - return this; - } - - private boolean stopIfInterrupted = false; /** * A special value for the resource that tells Spark to not try to process the app resource as a * file. This is useful when the class being executed is added to the application using other @@ -110,6 +101,13 @@ public SparkLauncher stopIfInterrupted() { static final Map launcherConfig = new HashMap<>(); + + private boolean stopIfInterrupted = false; + + /** Flag to decide on launching spark-submit as a child process or a thread **/ + private boolean launchSparkSubmitAsThread = false; + + /** * Set a configuration value for the launcher library. These config values do not affect the * launched application, but rather the behavior of the launcher library itself when managing @@ -123,6 +121,27 @@ public static void setConfig(String name, String value) { launcherConfig.put(name, value); } + /** + * Specifies that Spark Application be stopped if current process goes away. + * It tries stop/kill Spark Application if {@link LauncherServer} goes away. + * @return This launcher. + */ + public SparkLauncher stopIfInterrupted() { + this.stopIfInterrupted = true; + return this; + } + + /** + * Specifies that Spark Submit be launched as a daemon thread using reflection. + * Please note this feature is currently supported only for cluster deployment mode. + * + * @return This launcher. + */ + public SparkLauncher launchSparkSubmitAsThread(boolean launchSparkSubmitAsThread) { + this.launchSparkSubmitAsThread = launchSparkSubmitAsThread; + return this; + } + // Visible for testing. final SparkSubmitCommandBuilder builder; File workingDir; @@ -483,6 +502,27 @@ public Process launch() throws IOException { return childProc; } + private String getAppName() throws IOException { + String appName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); + if (appName == null) { + if (builder.appName != null) { + appName = builder.appName; + } else if (builder.mainClass != null) { + int dot = builder.mainClass.lastIndexOf("."); + if (dot >= 0 && dot < builder.mainClass.length() - 1) { + appName = builder.mainClass.substring(dot + 1, builder.mainClass.length()); + } else { + appName = builder.mainClass; + } + } else if (builder.appResource != null) { + appName = new File(builder.appResource).getName(); + } else { + appName = String.valueOf(COUNTER.incrementAndGet()); + } + } + return appName; + } + /** * Starts a Spark application. *

@@ -495,7 +535,7 @@ public Process launch() throws IOException { * that happens after that cannot be monitored. If the underlying application is launched as * a child process, {@link SparkAppHandle#kill()} can still be used to kill the child process. *

- * Currently, all applications are launched as child processes. The child's stdout and stderr + * If the applications are launched as child processes. The child's stdout and stderr * are merged and written to a logger (see java.util.logging) only if redirection * has not otherwise been configured on this SparkLauncher. The logger's name can be * defined by setting {@link #CHILD_PROCESS_LOGGER_NAME} in the app's configuration. If that @@ -504,11 +544,22 @@ public Process launch() throws IOException { * In all cases, the logger name will start with "org.apache.spark.launcher.app", to fit more * easily into the configuration of commonly-used logging systems. * + * If the applications are launched as a thread, the {@link SparkLauncher#redirectError} and + * {@link SparkLauncher#redirectToLog}, are not supported at this time. The existing process + * stdout and stderr will get all the log entries. + * * @since 1.6.0 * @param listeners Listeners to add to the handle before the app is launched. * @return A handle for the launched application. */ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) throws IOException { + if(launchSparkSubmitAsThread) { + return startApplicationAsThread(listeners); + } + return startApplicationAsChildProc(listeners); + } + + private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] listeners) throws IOException { ChildProcAppHandle handle = LauncherServer.newAppHandle(); for (SparkAppHandle.Listener l : listeners) { handle.addListener(l); @@ -535,49 +586,10 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr handle.kill(); throw ioe; } - return handle; } - private String getAppName() throws IOException { - String appName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); - if (appName == null) { - if (builder.appName != null) { - appName = builder.appName; - } else if (builder.mainClass != null) { - int dot = builder.mainClass.lastIndexOf("."); - if (dot >= 0 && dot < builder.mainClass.length() - 1) { - appName = builder.mainClass.substring(dot + 1, builder.mainClass.length()); - } else { - appName = builder.mainClass; - } - } else if (builder.appResource != null) { - appName = new File(builder.appResource).getName(); - } else { - appName = String.valueOf(COUNTER.incrementAndGet()); - } - } - return appName; - } - - /** - * Starts a Spark application. - *

- * This method returns a handle that provides information about the running application and can - * be used to do basic interaction with it. - *

- * The returned handle assumes that the application will instantiate a single SparkContext - * during its lifetime. Once that context reports a final state (one that indicates the - * SparkContext has stopped), the handle will not perform new state transitions, so anything - * that happens after that cannot be monitored. The underlying application is launched as - * a Thread, {@link SparkAppHandle#kill()} can still be used to kill the spark application. - *

- * @since 2.1.0 - * @param listeners Listeners to add to the handle before the app is launched. - * @return A handle for the launched application. - */ - public SparkAppHandle startApplicationInProcess(SparkAppHandle.Listener... listeners) throws IOException { - + private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... listeners) throws IOException { ChildThreadAppHandle handle = LauncherServer.newAppThreadHandle(); for (SparkAppHandle.Listener l : listeners) { handle.addListener(l); @@ -588,10 +600,12 @@ public SparkAppHandle startApplicationInProcess(SparkAppHandle.Listener... liste setConf(CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET, handle.getSecret()); setConf(CHILD_PROCESS_LAUNCHER_STOP_FLAG, String.valueOf(stopIfInterrupted)); try { - //trying to see if method is available in the classpath. + // It is important that spark-submit class is available in the classpath. + // Trying to see if method is available in the classpath else throws Exception. Method main = SparkSubmitRunner.getSparkSubmitMain(); Thread submitJobThread = new Thread(new SparkSubmitRunner(main, builder.buildSparkSubmitArgs())); submitJobThread.setName(appName); + submitJobThread.setDaemon(true); handle.setChildThread(submitJobThread); submitJobThread.start(); } catch (ClassNotFoundException cnfe) { @@ -600,10 +614,8 @@ public SparkAppHandle startApplicationInProcess(SparkAppHandle.Listener... liste throw new IOException(nsme); } return handle; - } - private ProcessBuilder createBuilder() { List cmd = new ArrayList<>(); String script = isWindows() ? "spark-submit.cmd" : "spark-submit"; @@ -679,6 +691,4 @@ protected void handleExtraArgs(List extra) { } - - } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java index eb421012bd5f..77d998a58f91 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java @@ -21,6 +21,13 @@ import java.lang.reflect.Method; import java.util.List; +/** + * This class used to by {@link SparkLauncher#startApplicationAsThread(SparkAppHandle.Listener...)} + *

+ * This class to start Spark applications programmatically as a Thread. This uses reflection and relies + * on find SparkSubmit main class. + *

+ */ class SparkSubmitRunner implements Runnable { private Method main; private final List args; @@ -29,18 +36,24 @@ class SparkSubmitRunner implements Runnable { this.main = main; this.args = args; } + /** - * Trying to see if method is available in the classpath. + * It is important that spark-submit class is available in the classpath. + * Trying to see if "org.apache.spark.deploy.SparkSubmit#Main" method is available in the classpath. + * + * @return Method to execute for starting Spark Application + * @throws ClassNotFoundException + * @throws NoSuchMethodException */ protected static Method getSparkSubmitMain() throws ClassNotFoundException, NoSuchMethodException { - Class cls = Class.forName("org.apache.spark.deploy.SparkSubmit"); - return cls.getDeclaredMethod("main", String[].class); + Class cls = Class.forName("org.apache.spark.deploy.SparkSubmit"); + return cls.getDeclaredMethod("main", String[].class); } @Override public void run() { try { - if(main == null) { + if (main == null) { main = getSparkSubmitMain(); } Object argsObj = args.toArray(new String[args.size()]); @@ -49,11 +62,10 @@ public void run() { throw new RuntimeException(illAcEx); } catch (InvocationTargetException invokEx) { throw new RuntimeException(invokEx); - } catch (RuntimeException re) { - throw re; - } catch (Exception e) { - throw new RuntimeException(e); + } catch (ClassNotFoundException cnfEx) { + throw new RuntimeException(cnfEx); + } catch (NoSuchMethodException nsmEx) { + throw new RuntimeException(nsmEx); } } - } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 2ab36aea38b8..a34057a6ca48 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -149,7 +149,7 @@ private[spark] class Client( def submitApplication(): ApplicationId = { var appId: ApplicationId = null try { - if ( launcherServerSecret !=null && launcherServerSecret != "" && launcherServerPort != 0) { + if (launcherServerSecret != null && launcherServerSecret != "" && launcherServerPort != 0) { launcherBackend.connect(launcherServerPort, launcherServerSecret, launcherServerStopFlag) } else { launcherBackend.connect() From eaa1bcab32ba89b8a38c0b6817c197af8fde0597 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 15 Sep 2016 13:56:20 -0500 Subject: [PATCH 08/50] Hide launcher internal configs from Environment page --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 ++++++ .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index ad50ea789a91..f7e00701b354 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -173,6 +173,12 @@ private[spark] class ApplicationMaster( sys.props.remove(e.key) } + // Clean up the spark launcher configuration so it doesn't show up in the Web UI for security. + sparkConf.remove("spark.launcher.internal.secret") + sparkConf.remove("spark.launcher.internal.port") + sys.props.remove("spark.launcher.internal.secret") + sys.props.remove("spark.launcher.internal.port") + resources.toMap } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a34057a6ca48..f89cf678a2d1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -659,6 +659,12 @@ private[spark] class Client( // by the AM. CACHE_CONFIGS.foreach(sparkConf.remove) + // Clear the spark-launcher-related entries from the configuration to avoid them polluting the + // UI's environment page. This works for client mode; for cluster mode, this is handled + // by the AM. + sparkConf.remove("spark.launcher.internal.secret") + sparkConf.remove("spark.launcher.internal.port") + localResources } From a3250acf96640568633a6b9270e1461b454aa313 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Mon, 3 Oct 2016 16:51:43 +0000 Subject: [PATCH 09/50] Avoid using System properties while launching Application in thread mode --- .../org/apache/spark/deploy/SparkSubmit.scala | 33 +++++++-- .../apache/spark/launcher/SparkLauncher.java | 3 + .../org/apache/spark/deploy/yarn/Client.scala | 70 ++++++++++++++----- 3 files changed, 85 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 7b6d5a394bc3..40068ad7c63b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -21,8 +21,10 @@ import java.io.{File, PrintStream} import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} import java.net.URL import java.security.PrivilegedExceptionAction +import java.util import scala.annotation.tailrec +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import org.apache.commons.lang3.StringUtils @@ -113,8 +115,8 @@ object SparkSubmit { } // scalastyle:on println - def main(args: Array[String]): Unit = { - val appArgs = new SparkSubmitArguments(args) + def main(argStrings: Array[String]) { + val appArgs = new SparkSubmitArguments(argStrings) if (appArgs.verbose) { // scalastyle:off println printStream.println(appArgs) @@ -676,10 +678,22 @@ object SparkSubmit { addJarToClasspath(jar, loader) } - for ((key, value) <- sysProps) { - System.setProperty(key, value) + val threadEnabled = sysProps.getOrElse("spark.launcher.internal.launcher.thread.enabled", + "false").toBoolean + if (!threadEnabled) { + for ((key, value) <- sysProps) { + System.setProperty(key, value) + } + } else { + for ( (key, value) <- sys.props ++ sys.env) { + if (!sysProps.contains(key)) { + sysProps.put(key, value) + } + } + } + var mainClass: Class[_] = null try { @@ -710,7 +724,12 @@ object SparkSubmit { printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.") } - val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) + val mainMethod = + if (threadEnabled ) { + mainClass.getMethods().filter(i => i.getName() == "mainWithEnv")(0) + } else { + mainClass.getMethod("main", new Array[String](0).getClass) + } if (!Modifier.isStatic(mainMethod.getModifiers)) { throw new IllegalStateException("The main method in the given main class must be static") } @@ -726,7 +745,11 @@ object SparkSubmit { } try { + if (threadEnabled) { + mainMethod.invoke(null, childArgs.toArray, sysProps) + } else { mainMethod.invoke(null, childArgs.toArray) + } } catch { case t: Throwable => findCause(t) match { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 574c1dc03b5a..9857ba61f81e 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -80,6 +80,8 @@ public class SparkLauncher { private static final String CHILD_PROCESS_LAUNCHER_STOP_FLAG = "spark.launcher.internal.stop.flag"; + private static final String CHILD_THREAD_LAUNCHER_ENABLED = "spark.launcher.internal.thread.enabled"; + /** * A special value for the resource that tells Spark to not try to process the app resource as a * file. This is useful when the class being executed is added to the application using other @@ -599,6 +601,7 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste setConf(LAUNCHER_INTERNAL_PORT,String.valueOf(LauncherServer.getServerInstance().getPort())); setConf(CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET, handle.getSecret()); setConf(CHILD_PROCESS_LAUNCHER_STOP_FLAG, String.valueOf(stopIfInterrupted)); + setConf(CHILD_THREAD_LAUNCHER_ENABLED,"true"); try { // It is important that spark-submit class is available in the classpath. // Trying to see if method is available in the classpath else throws Exception. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index f89cf678a2d1..ed44efb19cb3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -59,14 +59,24 @@ import org.apache.spark.util.Utils private[spark] class Client( val args: ClientArguments, val hadoopConf: Configuration, - val sparkConf: SparkConf) + val sparkConf: SparkConf, + val sysEnvironmentInput: Map[String, String]) extends Logging { import Client._ import YarnSparkHadoopUtil._ + def this(clientArgs: ClientArguments, spConf: SparkConf, sysenvironment: Map[String, String]) = + this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf, sysenvironment) + + def this(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) = + this(clientArgs, hadoopConf, spConf, Map() ++ sys.env) + def this(clientArgs: ClientArguments, spConf: SparkConf) = - this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) + this(clientArgs, spConf, Map() ++ sys.env) + + private val sysEnvironment: scala.collection.immutable.Map[scala.Predef.String, + scala.Predef.String] = collection.immutable.Map() ++ sysEnvironmentInput private val yarnClient = YarnClient.createYarnClient private val yarnConf = new YarnConfiguration(hadoopConf) @@ -77,7 +87,8 @@ private[spark] class Client( private val launcherServerSecret : String = sparkConf.get("spark.launcher.internal.secret", "") private val launcherServerStopFlag : Boolean = sparkConf.get("spark.launcher.internal.stop.flag", "false").toBoolean - + private val launcherServerInThreadFlag: Boolean = + sparkConf.get("spark.launcher.internal.thread.enabled", "false").toBoolean // AM related configurations private val amMemory = if (isClusterMode) { sparkConf.get(DRIVER_MEMORY).toInt @@ -704,7 +715,7 @@ private[spark] class Client( } Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey => - sys.env.get(envKey).foreach { path => + sysEnvironment.get(envKey).foreach { path => val dir = new File(path) if (dir.isDirectory()) { val files = dir.listFiles() @@ -757,7 +768,8 @@ private[spark] class Client( pySparkArchives: Seq[String]): HashMap[String, String] = { logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() - populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH)) + populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH), + sysEnvironment) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -775,7 +787,7 @@ private[spark] class Client( .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) } // Keep this for backwards compatibility but users should move to the config - sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + sysEnvironment.get("SPARK_YARN_USER_ENV").foreach { userEnvs => // Allow users to specify some environment variables. YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. @@ -804,7 +816,7 @@ private[spark] class Client( // Finally, update the Spark config to propagate PYTHONPATH to the AM and executors. if (pythonPath.nonEmpty) { - val pythonPathStr = (sys.env.get("PYTHONPATH") ++ pythonPath) + val pythonPathStr = (sysEnvironment.get("PYTHONPATH") ++ pythonPath) .mkString(YarnSparkHadoopUtil.getClassPathSeparator) env("PYTHONPATH") = pythonPathStr sparkConf.setExecutorEnv("PYTHONPATH", pythonPathStr) @@ -819,7 +831,7 @@ private[spark] class Client( // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). if (isClusterMode) { - sys.env.get("SPARK_JAVA_OPTS").foreach { value => + sysEnvironment.get("SPARK_JAVA_OPTS").foreach { value => val warning = s""" |SPARK_JAVA_OPTS was detected (set to '$value'). @@ -842,12 +854,12 @@ private[spark] class Client( // propagate PYSPARK_DRIVER_PYTHON and PYSPARK_PYTHON to driver in cluster mode Seq("PYSPARK_DRIVER_PYTHON", "PYSPARK_PYTHON").foreach { envname => if (!env.contains(envname)) { - sys.env.get(envname).foreach(env(envname) = _) + sysEnvironment.get(envname).foreach(env(envname) = _) } } } - sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp => + sysEnvironment.get(ENV_DIST_CLASSPATH).foreach { dcp => env(ENV_DIST_CLASSPATH) = dcp } @@ -913,7 +925,8 @@ private[spark] class Client( // Include driver-specific java options if we are launching a driver if (isClusterMode) { - val driverOpts = sparkConf.get(DRIVER_JAVA_OPTIONS).orElse(sys.env.get("SPARK_JAVA_OPTS")) + val driverOpts = sparkConf.get(DRIVER_JAVA_OPTIONS).orElse( + sysEnvironment.get("SPARK_JAVA_OPTS")) driverOpts.foreach { opts => javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) } @@ -1185,10 +1198,10 @@ private[spark] class Client( } private def findPySparkArchives(): Seq[String] = { - sys.env.get("PYSPARK_ARCHIVES_PATH") + sysEnvironment.get("PYSPARK_ARCHIVES_PATH") .map(_.split(",").toSeq) .getOrElse { - val pyLibPath = Seq(sys.env("SPARK_HOME"), "python", "lib").mkString(File.separator) + val pyLibPath = Seq(sysEnvironment("SPARK_HOME"), "python", "lib").mkString(File.separator) val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.") @@ -1204,6 +1217,10 @@ private[spark] class Client( private object Client extends Logging { def main(argStrings: Array[String]) { + mainWithEnv(argStrings, Map() ++ sys.env) + } + + def mainWithEnv(argStrings: Array[String], env: Map[String, String]): Unit = { if (!sys.props.contains("SPARK_SUBMIT")) { logWarning("WARNING: This client is deprecated and will be removed in a " + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") @@ -1212,12 +1229,22 @@ private object Client extends Logging { // Set an env variable indicating we are running in YARN mode. // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") + + val threadEnabled: Boolean = env.get("spark.launcher.internal.thread.enabled"). + getOrElse("false").toBoolean + val sparkConf = new SparkConf + if( threadEnabled) { + for ((key, value) <- env if key.startsWith("spark.")) { + sparkConf.set(key, value, true) + } + } val args = new ClientArguments(argStrings) - new Client(args, sparkConf).run() + new Client(args, sparkConf, env).run() } + // Alias for the user jar val APP_JAR_NAME: String = "__app__.jar" @@ -1322,6 +1349,15 @@ private object Client extends Logging { triedDefault.toOption } + private[yarn] def populateClasspath( + args: ClientArguments, + conf: Configuration, + sparkConf: SparkConf, + env: HashMap[String, String], + extraClassPath: Option[String]): Unit = { + populateClasspath(args, conf, sparkConf, env, extraClassPath, sys.env) + } + /** * Populate the classpath entry in the given environment map. * @@ -1337,7 +1373,9 @@ private object Client extends Logging { conf: Configuration, sparkConf: SparkConf, env: HashMap[String, String], - extraClassPath: Option[String] = None): Unit = { + extraClassPath: Option[String] = None, + providedEnv: scala.collection.immutable.Map[scala.Predef.String, scala.Predef.String] + = sys.env): Unit = { extraClassPath.foreach { cp => addClasspathEntry(getClusterPath(sparkConf, cp), env) } @@ -1383,7 +1421,7 @@ private object Client extends Logging { } populateHadoopClasspath(conf, env) - sys.env.get(ENV_DIST_CLASSPATH).foreach { cp => + providedEnv.get(ENV_DIST_CLASSPATH).foreach { cp => addClasspathEntry(getClusterPath(sparkConf, cp), env) } } From 58c6bac485d77853c912cd220e34173b5e945d1d Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Tue, 18 Oct 2016 16:39:10 +0000 Subject: [PATCH 10/50] Addressing few formatting related comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +++----- .../apache/spark/launcher/SparkLauncher.java | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +++---- .../org/apache/spark/deploy/yarn/Client.scala | 25 ++++++++----------- 4 files changed, 19 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 40068ad7c63b..c6df5c4b5928 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -115,8 +115,8 @@ object SparkSubmit { } // scalastyle:on println - def main(argStrings: Array[String]) { - val appArgs = new SparkSubmitArguments(argStrings) + def main(args: Array[String]): Unit = { + val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { // scalastyle:off println printStream.println(appArgs) @@ -693,7 +693,6 @@ object SparkSubmit { } - var mainClass: Class[_] = null try { @@ -724,8 +723,7 @@ object SparkSubmit { printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.") } - val mainMethod = - if (threadEnabled ) { + val mainMethod = if (threadEnabled ) { mainClass.getMethods().filter(i => i.getName() == "mainWithEnv")(0) } else { mainClass.getMethod("main", new Array[String](0).getClass) @@ -748,7 +746,7 @@ object SparkSubmit { if (threadEnabled) { mainMethod.invoke(null, childArgs.toArray, sysProps) } else { - mainMethod.invoke(null, childArgs.toArray) + mainMethod.invoke(null, childArgs.toArray) } } catch { case t: Throwable => diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 9857ba61f81e..1a1624d6d787 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -555,7 +555,7 @@ private String getAppName() throws IOException { * @return A handle for the launched application. */ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) throws IOException { - if(launchSparkSubmitAsThread) { + if (launchSparkSubmitAsThread) { return startApplicationAsThread(listeners); } return startApplicationAsChildProc(listeners); diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index f7e00701b354..92000eec2300 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -174,11 +174,10 @@ private[spark] class ApplicationMaster( } // Clean up the spark launcher configuration so it doesn't show up in the Web UI for security. - sparkConf.remove("spark.launcher.internal.secret") - sparkConf.remove("spark.launcher.internal.port") - sys.props.remove("spark.launcher.internal.secret") - sys.props.remove("spark.launcher.internal.port") - + Seq("spark.launcher.internal.secret", "spark.launcher.internal.port").foreach { e => + sparkConf.remove(e) + sys.props.remove(e) + } resources.toMap } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index ed44efb19cb3..8e5e69ac2bf3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -66,14 +66,14 @@ private[spark] class Client( import Client._ import YarnSparkHadoopUtil._ - def this(clientArgs: ClientArguments, spConf: SparkConf, sysenvironment: Map[String, String]) = - this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf, sysenvironment) + def this(clientArgs: ClientArguments, spConf: SparkConf, sysEnv: Map[String, String]) = + this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf, sysEnv) def this(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) = this(clientArgs, hadoopConf, spConf, Map() ++ sys.env) def this(clientArgs: ClientArguments, spConf: SparkConf) = - this(clientArgs, spConf, Map() ++ sys.env) + this(clientArgs, spConf, Map() ++ sys.env) private val sysEnvironment: scala.collection.immutable.Map[scala.Predef.String, scala.Predef.String] = collection.immutable.Map() ++ sysEnvironmentInput @@ -665,17 +665,13 @@ private[spark] class Client( remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE, LOCALIZED_CONF_DIR, statCache, appMasterOnly = false) - // Clear the cache-related entries from the configuration to avoid them polluting the - // UI's environment page. This works for client mode; for cluster mode, this is handled - // by the AM. + // Clear the cache-related and spark-launcher entries from the configuration to avoid them + // polluting the UI's environment page. This works for client mode; for cluster mode, this + // is handled by the AM. CACHE_CONFIGS.foreach(sparkConf.remove) - - // Clear the spark-launcher-related entries from the configuration to avoid them polluting the - // UI's environment page. This works for client mode; for cluster mode, this is handled - // by the AM. - sparkConf.remove("spark.launcher.internal.secret") - sparkConf.remove("spark.launcher.internal.port") - + Seq("spark.launcher.internal.secret", "spark.launcher.internal.port").foreach { e => + sparkConf.remove(e) + } localResources } @@ -1234,7 +1230,7 @@ private object Client extends Logging { getOrElse("false").toBoolean val sparkConf = new SparkConf - if( threadEnabled) { + if (threadEnabled) { for ((key, value) <- env if key.startsWith("spark.")) { sparkConf.set(key, value, true) } @@ -1244,7 +1240,6 @@ private object Client extends Logging { new Client(args, sparkConf, env).run() } - // Alias for the user jar val APP_JAR_NAME: String = "__app__.jar" From 0a57684f589c2ea4ea7613a60082a9418ee0f953 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 19 Oct 2016 20:54:54 +0000 Subject: [PATCH 11/50] Adding SparkApp Trait --- .../scala/org/apache/spark/SparkApp.scala | 28 +++++++++++++++++++ .../org/apache/spark/deploy/SparkSubmit.scala | 4 +-- .../org/apache/spark/deploy/yarn/Client.scala | 22 +++++++-------- 3 files changed, 40 insertions(+), 14 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/SparkApp.scala diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala new file mode 100644 index 000000000000..273fc9fca0f6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkApp.scala @@ -0,0 +1,28 @@ +/* + * 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.spark + +/** + * A client that SparkSubmit uses to launch spark Application. + * This is currently supported only in YARN mode. + */ +trait SparkApp { + + def sparkMain(args: Array[String], conf: scala.collection.mutable.Map[String, String]): Unit + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c6df5c4b5928..4333040d02aa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -685,7 +685,7 @@ object SparkSubmit { System.setProperty(key, value) } } else { - for ( (key, value) <- sys.props ++ sys.env) { + for ( (key, value) <- sys.env) { if (!sysProps.contains(key)) { sysProps.put(key, value) } @@ -724,7 +724,7 @@ object SparkSubmit { } val mainMethod = if (threadEnabled ) { - mainClass.getMethods().filter(i => i.getName() == "mainWithEnv")(0) + mainClass.getMethods().filter(i => i.getName() == "sparkMain")(0) } else { mainClass.getMethod("main", new Array[String](0).getClass) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 8e5e69ac2bf3..55c9f49416f1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import java.io.{File, FileOutputStream, IOException, OutputStreamWriter} -import java.net.{InetAddress, UnknownHostException, URI} +import java.net.{InetAddress, URI, UnknownHostException} import java.nio.ByteBuffer import java.nio.charset.StandardCharsets import java.util.{Properties, UUID} @@ -28,7 +28,6 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.util.{Failure, Success, Try} import scala.util.control.NonFatal - import com.google.common.base.Objects import com.google.common.io.Files import org.apache.hadoop.conf.Configuration @@ -46,8 +45,7 @@ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records - -import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException} +import org.apache.spark.{SecurityManager, SparkApp, SparkConf, SparkContext, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager @@ -56,6 +54,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} import org.apache.spark.util.Utils + private[spark] class Client( val args: ClientArguments, val hadoopConf: Configuration, @@ -1210,13 +1209,13 @@ private[spark] class Client( } -private object Client extends Logging { +private object Client extends SparkApp with Logging { def main(argStrings: Array[String]) { - mainWithEnv(argStrings, Map() ++ sys.env) + sparkMain(argStrings, Map() ++ sys.env) } - def mainWithEnv(argStrings: Array[String], env: Map[String, String]): Unit = { + override def sparkMain(args: Array[String], conf: Map[String, String]): Unit = { if (!sys.props.contains("SPARK_SUBMIT")) { logWarning("WARNING: This client is deprecated and will be removed in a " + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") @@ -1226,18 +1225,18 @@ private object Client extends Logging { // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val threadEnabled: Boolean = env.get("spark.launcher.internal.thread.enabled"). + val threadEnabled: Boolean = conf.get("spark.launcher.internal.thread.enabled"). getOrElse("false").toBoolean val sparkConf = new SparkConf if (threadEnabled) { - for ((key, value) <- env if key.startsWith("spark.")) { + for ((key, value) <- conf if key.startsWith("spark.")) { sparkConf.set(key, value, true) } } - val args = new ClientArguments(argStrings) - new Client(args, sparkConf, env).run() + val argsForClient = new ClientArguments(args) + new Client(argsForClient, sparkConf, conf).run() } // Alias for the user jar @@ -1574,5 +1573,4 @@ private object Client extends Logging { def isLocalUri(uri: String): Boolean = { uri.startsWith(s"$LOCAL_SCHEME:") } - } From 1fe498b5bc5ab903d93fd6ec278a5358059d1d90 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 19 Oct 2016 22:05:51 +0000 Subject: [PATCH 12/50] Adding documentation to launcher package --- .../apache/spark/launcher/package-info.java | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java index d1ac39bdc76a..d231fdb0a349 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -49,6 +49,38 @@ * * *

+ * Currently, for currently while launching spark application with + * {@link org.apache.spark.launcher.SparkLauncher#startApplication( + * org.apache.spark.launcher.SparkAppHandle.Listener...)}, there are two options available + * for YARN manager in cluster deploy mode: + * - to launch Spark Application as a Thread inside current JVM using + * the {@link org.apache.spark.launcher.SparkLauncher#launchSparkSubmitAsThread(boolean)} + * - to request application be killed if launcher process exits using + * the {@link org.apache.spark.launcher.SparkLauncher#stopIfInterrupted()}. + *

+ * + *
+ * {@code
+ *   import org.apache.spark.launcher.SparkAppHandle;
+ *   import org.apache.spark.launcher.SparkLauncher;
+ *
+ *   public class MyLauncher {
+ *     public static void main(String[] args) throws Exception {
+ *       SparkAppHandle handle = new SparkLauncher()
+ *         .setAppResource("/my/app.jar")
+ *         .setMainClass("my.spark.app.Main")
+ *         .setMaster("yarn")
+ *         .stopIfInterrupted()
+ *         .launchSparkSubmitAsThread(true)
+ *         .setConf(SparkLauncher.DRIVER_MEMORY, "2g")
+ *         .startApplication();
+ *       // Use handle API to monitor / control application.
+ *     }
+ *   }
+ * }
+ * 
+ * + *

* It's also possible to launch a raw child process, using the * {@link org.apache.spark.launcher.SparkLauncher#launch()} method: *

From 25c3258d9d1c67b308d44578045dcd3ab9706c2d Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 19 Oct 2016 22:26:39 +0000 Subject: [PATCH 13/50] Another minor code comment fix --- yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 55c9f49416f1..d345214d8643 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -86,8 +86,6 @@ private[spark] class Client( private val launcherServerSecret : String = sparkConf.get("spark.launcher.internal.secret", "") private val launcherServerStopFlag : Boolean = sparkConf.get("spark.launcher.internal.stop.flag", "false").toBoolean - private val launcherServerInThreadFlag: Boolean = - sparkConf.get("spark.launcher.internal.thread.enabled", "false").toBoolean // AM related configurations private val amMemory = if (isClusterMode) { sparkConf.get(DRIVER_MEMORY).toInt @@ -1368,7 +1366,7 @@ private object Client extends SparkApp with Logging { sparkConf: SparkConf, env: HashMap[String, String], extraClassPath: Option[String] = None, - providedEnv: scala.collection.immutable.Map[scala.Predef.String, scala.Predef.String] + providedEnv: scala.collection.immutable.Map[String, String] = sys.env): Unit = { extraClassPath.foreach { cp => addClasspathEntry(getClusterPath(sparkConf, cp), env) From 14050f5d16b01d5081b456b92afbe78cd79b8d25 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 19 Oct 2016 22:48:47 +0000 Subject: [PATCH 14/50] Fixing scala style issues --- core/src/main/scala/org/apache/spark/SparkApp.scala | 6 +++--- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 4 +++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala index 273fc9fca0f6..c3b9bd92344d 100644 --- a/core/src/main/scala/org/apache/spark/SparkApp.scala +++ b/core/src/main/scala/org/apache/spark/SparkApp.scala @@ -18,9 +18,9 @@ package org.apache.spark /** - * A client that SparkSubmit uses to launch spark Application. - * This is currently supported only in YARN mode. - */ + * A client that SparkSubmit uses to launch spark Application. + * This is currently supported only in YARN mode. + */ trait SparkApp { def sparkMain(args: Array[String], conf: scala.collection.mutable.Map[String, String]): Unit diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index d345214d8643..a0a7a8a86406 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import java.io.{File, FileOutputStream, IOException, OutputStreamWriter} -import java.net.{InetAddress, URI, UnknownHostException} +import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer import java.nio.charset.StandardCharsets import java.util.{Properties, UUID} @@ -28,6 +28,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.util.{Failure, Success, Try} import scala.util.control.NonFatal + import com.google.common.base.Objects import com.google.common.io.Files import org.apache.hadoop.conf.Configuration @@ -45,6 +46,7 @@ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records + import org.apache.spark.{SecurityManager, SparkApp, SparkConf, SparkContext, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ From 92e444578ba5dfea02f473101999a82fa8c8d325 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 20 Oct 2016 00:48:07 +0000 Subject: [PATCH 15/50] Make SparkApp trait Singleton --- core/src/main/scala/org/apache/spark/SparkApp.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala index c3b9bd92344d..fa5e7423eb89 100644 --- a/core/src/main/scala/org/apache/spark/SparkApp.scala +++ b/core/src/main/scala/org/apache/spark/SparkApp.scala @@ -22,6 +22,7 @@ package org.apache.spark * This is currently supported only in YARN mode. */ trait SparkApp { + this: Singleton => def sparkMain(args: Array[String], conf: scala.collection.mutable.Map[String, String]): Unit From 64a21b3cc15bc1887e3d40d839bbeb1e105378c3 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 28 Oct 2016 16:53:17 -0500 Subject: [PATCH 16/50] Addressing Review comments --- .../scala/org/apache/spark/SparkApp.scala | 7 ++- .../org/apache/spark/deploy/SparkSubmit.scala | 41 +++++++++------- .../spark/launcher/LauncherBackend.scala | 18 +++---- .../launcher/AbstractSparkAppHandle.java | 3 +- .../spark/launcher/ChildThreadAppHandle.java | 1 + .../apache/spark/launcher/SparkLauncher.java | 38 +++++++------- .../apache/spark/launcher/package-info.java | 10 ++-- .../org/apache/spark/deploy/yarn/Client.scala | 49 ++++++++++--------- .../org/apache/spark/deploy/yarn/config.scala | 5 ++ 9 files changed, 92 insertions(+), 80 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala index fa5e7423eb89..71986f711b06 100644 --- a/core/src/main/scala/org/apache/spark/SparkApp.scala +++ b/core/src/main/scala/org/apache/spark/SparkApp.scala @@ -21,9 +21,12 @@ package org.apache.spark * A client that SparkSubmit uses to launch spark Application. * This is currently supported only in YARN mode. */ -trait SparkApp { +private[spark] trait SparkApp { this: Singleton => - def sparkMain(args: Array[String], conf: scala.collection.mutable.Map[String, String]): Unit + def sparkMain( + args: Array[String], + conf: scala.collection.immutable.Map[String, String], + sysConf: scala.collection.immutable.Map[String, String]): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4333040d02aa..dcc4324d34ba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -44,6 +44,7 @@ import org.apache.ivy.plugins.resolver.{ChainResolver, FileSystemResolver, IBibl import org.apache.spark.{SPARK_REVISION, SPARK_VERSION, SparkException, SparkUserAppException} import org.apache.spark.{SPARK_BRANCH, SPARK_BUILD_DATE, SPARK_BUILD_USER, SPARK_REPO_URL} +import org.apache.spark.SparkApp import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ import org.apache.spark.launcher.SparkLauncher @@ -678,20 +679,8 @@ object SparkSubmit { addJarToClasspath(jar, loader) } - val threadEnabled = sysProps.getOrElse("spark.launcher.internal.launcher.thread.enabled", - "false").toBoolean - if (!threadEnabled) { - for ((key, value) <- sysProps) { - System.setProperty(key, value) - } - } else { - for ( (key, value) <- sys.env) { - if (!sysProps.contains(key)) { - sysProps.put(key, value) - } - } - - } + val threadEnabled = sysProps.getOrElse(SparkLauncher.LAUNCHER_INTERNAL_THREAD_ENABLED, + "false").toBoolean var mainClass: Class[_] = null @@ -723,8 +712,24 @@ object SparkSubmit { printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.") } - val mainMethod = if (threadEnabled ) { - mainClass.getMethods().filter(i => i.getName() == "sparkMain")(0) + val isSparkApp = mainClass.getMethods().filter(_.getName() == "sparkMain").length > 0 + + val childSparkConf = sysProps.filter( p => p._1.startsWith("spark.")).toMap + val childSysProps = sys.env + if (isSparkApp && threadEnabled) { + sys.env.foreach { case (key, value) => + if (!sysProps.contains(key)) { + sysProps.put(key, value) + } + } + } else { + sysProps.foreach { case (key, value) => + System.setProperty(key, value) + } + } + + val mainMethod = if (isSparkApp) { + mainClass.getMethods().filter(_.getName() == "sparkMain")(0) } else { mainClass.getMethod("main", new Array[String](0).getClass) } @@ -743,8 +748,8 @@ object SparkSubmit { } try { - if (threadEnabled) { - mainMethod.invoke(null, childArgs.toArray, sysProps) + if (isSparkApp) { + mainMethod.invoke(null, childArgs.toArray, childSparkConf, childSysProps) } else { mainMethod.invoke(null, childArgs.toArray) } diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala index 825750b650cc..499c3380fdb6 100644 --- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala +++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala @@ -36,7 +36,7 @@ private[spark] abstract class LauncherBackend extends Logging { private var clientThread: Thread = _ private var connection: BackendConnection = _ private var lastState: SparkAppHandle.State = _ - private var stopFlag: Boolean = false + private var stopOnShutdown: Boolean = false @volatile private var _isConnected = false def connect(): Unit = { @@ -44,11 +44,7 @@ private[spark] abstract class LauncherBackend extends Logging { val secret = sys.env.get(LauncherProtocol.ENV_LAUNCHER_SECRET) val stopFlag = sys.env.get(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG).map(_.toBoolean) if (port != None && secret != None) { - if (stopFlag != None) { - connect(port.get, secret.get, stopFlag.get) - } else { - connect(port.get, secret.get) - } + connect(port.get, secret.get, stopFlag.getOrElse(false)) } } @@ -59,13 +55,13 @@ private[spark] abstract class LauncherBackend extends Logging { clientThread = LauncherBackend.threadFactory.newThread(connection) clientThread.start() _isConnected = true - if (stopFlag) { + if (stopOnShutdown) { logDebug("Adding shutdown hook") // force eager creation of logger var _shutdownHookRef = ShutdownHookManager.addShutdownHook( ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () => logInfo("Invoking onStopRequest() from shutdown hook") try { - if (_isConnected && stopFlag) { + if (_isConnected && stopOnShutdown) { onStopRequest() } } @@ -78,7 +74,7 @@ private[spark] abstract class LauncherBackend extends Logging { } def connect(port: Int, secret: String, stopFlag: Boolean): Unit = { - this.stopFlag = stopFlag + this.stopOnShutdown = stopFlag connect(port, secret) } @@ -104,7 +100,7 @@ private[spark] abstract class LauncherBackend extends Logging { if (connection != null && lastState != state) { connection.send(new SetState(state)) lastState = state - if (!_isConnected && stopFlag) { + if (!_isConnected && stopOnShutdown) { fireStopRequest() } } @@ -146,7 +142,7 @@ private[spark] abstract class LauncherBackend extends Logging { override def close(): Unit = { try { super.close() - if (stopFlag) { + if (stopOnShutdown) { fireStopRequest() } } finally { diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java index c1a83f67789f..97287b34d2e2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java @@ -18,15 +18,14 @@ package org.apache.spark.launcher; import java.io.IOException; -import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.logging.Level; import java.util.logging.Logger; - public abstract class AbstractSparkAppHandle implements SparkAppHandle { private static final Logger LOG = Logger.getLogger(AbstractSparkAppHandle.class.getName()); + protected final String secret; protected final LauncherServer server; protected boolean disposed; diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java index c216fe1f4eae..eafdb5411d73 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java @@ -21,6 +21,7 @@ public class ChildThreadAppHandle extends AbstractSparkAppHandle { private static final Logger LOG = Logger.getLogger(ChildThreadAppHandle.class.getName()); + private Thread childThread; public ChildThreadAppHandle(String secret, LauncherServer server) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 1a1624d6d787..1d031073d97b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -74,13 +74,13 @@ public class SparkLauncher { /** Logger name to use when launching a child process. */ public static final String CHILD_PROCESS_LOGGER_NAME = "spark.launcher.childProcLoggerName"; - private static final String LAUNCHER_INTERNAL_PORT = "spark.launcher.internal.port"; + public static final String LAUNCHER_INTERNAL_PORT = "spark.launcher.internal.port"; - private static final String CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET = "spark.launcher.internal.secret"; + public static final String LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET = "spark.launcher.internal.secret"; - private static final String CHILD_PROCESS_LAUNCHER_STOP_FLAG = "spark.launcher.internal.stop.flag"; + public static final String LAUNCHER_INTERNAL_STOP_IF_SHUTDOWN = "spark.launcher.internal.stop.if.shutdown"; - private static final String CHILD_THREAD_LAUNCHER_ENABLED = "spark.launcher.internal.thread.enabled"; + public static final String LAUNCHER_INTERNAL_THREAD_ENABLED = "spark.launcher.internal.thread.enabled"; /** * A special value for the resource that tells Spark to not try to process the app resource as a @@ -104,10 +104,10 @@ public class SparkLauncher { static final Map launcherConfig = new HashMap<>(); - private boolean stopIfInterrupted = false; + private boolean stopIfLauncherShutdown = false; /** Flag to decide on launching spark-submit as a child process or a thread **/ - private boolean launchSparkSubmitAsThread = false; + private boolean launchAsThread = false; /** @@ -128,19 +128,19 @@ public static void setConfig(String name, String value) { * It tries stop/kill Spark Application if {@link LauncherServer} goes away. * @return This launcher. */ - public SparkLauncher stopIfInterrupted() { - this.stopIfInterrupted = true; + public SparkLauncher stopIfLauncherShutdown() { + this.stopIfLauncherShutdown = true; return this; } /** * Specifies that Spark Submit be launched as a daemon thread using reflection. - * Please note this feature is currently supported only for cluster deployment mode. + * Please note this feature is currently supported only for YARN cluster deployment mode. * * @return This launcher. */ - public SparkLauncher launchSparkSubmitAsThread(boolean launchSparkSubmitAsThread) { - this.launchSparkSubmitAsThread = launchSparkSubmitAsThread; + public SparkLauncher launchAsThread(boolean launchAsThread) { + this.launchAsThread = launchAsThread; return this; } @@ -555,7 +555,7 @@ private String getAppName() throws IOException { * @return A handle for the launched application. */ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) throws IOException { - if (launchSparkSubmitAsThread) { + if (launchAsThread) { return startApplicationAsThread(listeners); } return startApplicationAsChildProc(listeners); @@ -581,7 +581,7 @@ private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] lis pb.environment().put(LauncherProtocol.ENV_LAUNCHER_PORT, String.valueOf(LauncherServer.getServerInstance().getPort())); pb.environment().put(LauncherProtocol.ENV_LAUNCHER_SECRET, handle.getSecret()); - pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG, String.valueOf(stopIfInterrupted)); + pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG, String.valueOf(stopIfLauncherShutdown)); try { handle.setChildProc(pb.start(), loggerName); } catch (IOException ioe) { @@ -598,10 +598,10 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste } String appName = getAppName(); - setConf(LAUNCHER_INTERNAL_PORT,String.valueOf(LauncherServer.getServerInstance().getPort())); - setConf(CHILD_PROCESS_LAUNCHER_INTERNAL_SECRET, handle.getSecret()); - setConf(CHILD_PROCESS_LAUNCHER_STOP_FLAG, String.valueOf(stopIfInterrupted)); - setConf(CHILD_THREAD_LAUNCHER_ENABLED,"true"); + setConf(LAUNCHER_INTERNAL_PORT, String.valueOf(LauncherServer.getServerInstance().getPort())); + setConf(LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, handle.getSecret()); + setConf(LAUNCHER_INTERNAL_STOP_IF_SHUTDOWN, String.valueOf(stopIfLauncherShutdown)); + setConf(LAUNCHER_INTERNAL_THREAD_ENABLED, "true"); try { // It is important that spark-submit class is available in the classpath. // Trying to see if method is available in the classpath else throws Exception. @@ -612,9 +612,9 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste handle.setChildThread(submitJobThread); submitJobThread.start(); } catch (ClassNotFoundException cnfe) { - throw new IOException(cnfe); + throw new IOException("Please make sure spark-core is in the classpath.", cnfe); } catch (NoSuchMethodException nsme) { - throw new IOException(nsme); + throw new IOException("Please make sure spark-core version is correct.", nsme); } return handle; } diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java index d231fdb0a349..dd53e61a6558 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -54,15 +54,15 @@ * org.apache.spark.launcher.SparkAppHandle.Listener...)}, there are two options available * for YARN manager in cluster deploy mode: * - to launch Spark Application as a Thread inside current JVM using - * the {@link org.apache.spark.launcher.SparkLauncher#launchSparkSubmitAsThread(boolean)} + * the {@link org.apache.spark.launcher.SparkLauncher#launchAsThread(boolean)} * - to request application be killed if launcher process exits using - * the {@link org.apache.spark.launcher.SparkLauncher#stopIfInterrupted()}. + * the {@link org.apache.spark.launcher.SparkLauncher#stopIfLauncherShutdown()}. *

* *
  * {@code
  *   import org.apache.spark.launcher.SparkAppHandle;
- *   import org.apache.spark.launcher.SparkLauncher;
+ *   import org.apache.spark.launcher.SparkLauncher;`
  *
  *   public class MyLauncher {
  *     public static void main(String[] args) throws Exception {
@@ -70,8 +70,8 @@
  *         .setAppResource("/my/app.jar")
  *         .setMainClass("my.spark.app.Main")
  *         .setMaster("yarn")
- *         .stopIfInterrupted()
- *         .launchSparkSubmitAsThread(true)
+ *         .stopIfLauncherShutdown()
+ *         .launchAsThread(true)
  *         .setConf(SparkLauncher.DRIVER_MEMORY, "2g")
  *         .startApplication();
  *       // Use handle API to monitor / control application.
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index a0a7a8a86406..9a62e9a81357 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -53,7 +53,7 @@ import org.apache.spark.deploy.yarn.config._
 import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
-import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
+import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, SparkLauncher, YarnCommandBuilderUtils}
 import org.apache.spark.util.Utils
 
 
@@ -61,33 +61,35 @@ private[spark] class Client(
     val args: ClientArguments,
     val hadoopConf: Configuration,
     val sparkConf: SparkConf,
-    val sysEnvironmentInput: Map[String, String])
+    val sysEnvironment: scala.collection.immutable.Map[String, String])
   extends Logging {
 
   import Client._
   import YarnSparkHadoopUtil._
 
-  def this(clientArgs: ClientArguments, spConf: SparkConf, sysEnv: Map[String, String]) =
+  def this(
+    clientArgs: ClientArguments,
+    spConf: SparkConf,
+    sysEnv: scala.collection.immutable.Map[String, String]) =
     this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf, sysEnv)
 
   def this(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) =
-    this(clientArgs, hadoopConf, spConf, Map() ++ sys.env)
+    this(clientArgs, hadoopConf, spConf, sys.env.toMap)
 
   def this(clientArgs: ClientArguments, spConf: SparkConf) =
-    this(clientArgs, spConf, Map() ++ sys.env)
-
-  private val sysEnvironment: scala.collection.immutable.Map[scala.Predef.String,
-   scala.Predef.String] = collection.immutable.Map() ++ sysEnvironmentInput
+    this(clientArgs, spConf, sys.env.toMap)
 
   private val yarnClient = YarnClient.createYarnClient
   private val yarnConf = new YarnConfiguration(hadoopConf)
 
   private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster"
 
-  private val launcherServerPort : Int = sparkConf.get("spark.launcher.internal.port", "0").toInt
-  private val launcherServerSecret : String = sparkConf.get("spark.launcher.internal.secret", "")
-  private val launcherServerStopFlag : Boolean = sparkConf.get("spark.launcher.internal.stop.flag",
-    "false").toBoolean
+  private val launcherServerPort : Int =
+    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_PORT, "0").toInt
+  private val launcherServerSecret : String =
+    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, "")
+  private val launcherServerStopIfShutdown : Boolean =
+    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_STOP_IF_SHUTDOWN, "false").toBoolean
   // AM related configurations
   private val amMemory = if (isClusterMode) {
     sparkConf.get(DRIVER_MEMORY).toInt
@@ -160,7 +162,10 @@ private[spark] class Client(
     var appId: ApplicationId = null
     try {
       if (launcherServerSecret != null && launcherServerSecret != "" && launcherServerPort != 0) {
-        launcherBackend.connect(launcherServerPort, launcherServerSecret, launcherServerStopFlag)
+        launcherBackend.connect(
+          launcherServerPort,
+          launcherServerSecret,
+          launcherServerStopIfShutdown)
       } else {
         launcherBackend.connect()
       }
@@ -668,9 +673,8 @@ private[spark] class Client(
     // polluting the UI's environment page. This works for client mode; for cluster mode, this
     // is handled by the AM.
     CACHE_CONFIGS.foreach(sparkConf.remove)
-    Seq("spark.launcher.internal.secret", "spark.launcher.internal.port").foreach { e =>
-      sparkConf.remove(e)
-    }
+    LAUNCHER_CONFIGs.foreach(sparkConf.remove)
+
     localResources
   }
 
@@ -1211,11 +1215,10 @@ private[spark] class Client(
 
 private object Client extends SparkApp with Logging {
 
-  def main(argStrings: Array[String]) {
-    sparkMain(argStrings, Map() ++ sys.env)
-  }
-
-  override def sparkMain(args: Array[String], conf: Map[String, String]): Unit = {
+  override def sparkMain(
+    args: Array[String],
+    conf: scala.collection.immutable.Map[String, String],
+    sysConf: scala.collection.immutable.Map[String, String]): Unit = {
     if (!sys.props.contains("SPARK_SUBMIT")) {
       logWarning("WARNING: This client is deprecated and will be removed in a " +
         "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"")
@@ -1225,7 +1228,7 @@ private object Client extends SparkApp with Logging {
     // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes
     System.setProperty("SPARK_YARN_MODE", "true")
 
-    val threadEnabled: Boolean = conf.get("spark.launcher.internal.thread.enabled").
+    val threadEnabled: Boolean = conf.get(SparkLauncher.LAUNCHER_INTERNAL_THREAD_ENABLED).
       getOrElse("false").toBoolean
 
     val sparkConf = new SparkConf
@@ -1236,7 +1239,7 @@ private object Client extends SparkApp with Logging {
     }
 
     val argsForClient = new ClientArguments(args)
-    new Client(argsForClient, sparkConf, conf).run()
+    new Client(argsForClient, sparkConf, sysConf).run()
   }
 
   // Alias for the user jar
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
index ca8c89043aa8..20fe54019601 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -20,6 +20,7 @@ package org.apache.spark.deploy.yarn
 import java.util.concurrent.TimeUnit
 
 import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.launcher.SparkLauncher
 import org.apache.spark.network.util.ByteUnit
 
 package object config {
@@ -339,4 +340,8 @@ package object config {
     CACHED_FILES_TYPES,
     CACHED_CONF_ARCHIVE)
 
+  private[yarn] val LAUNCHER_CONFIGs = Seq(
+    SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET,
+    SparkLauncher.LAUNCHER_INTERNAL_PORT)
+
 }

From c207b309329b6b9b45dc4ffac6fada8befe973b9 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Mon, 7 Nov 2016 21:11:33 +0000
Subject: [PATCH 17/50] Fix review  minor comments

---
 .../org/apache/spark/launcher/ChildProcAppHandle.java  |  7 ++++++-
 .../java/org/apache/spark/launcher/SparkLauncher.java  |  3 +++
 .../org/apache/spark/launcher/SparkSubmitRunner.java   | 10 ++--------
 3 files changed, 11 insertions(+), 9 deletions(-)

diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
index f7bcc1c1b25c..40adb5b5b6e3 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
@@ -49,7 +49,12 @@ public synchronized void kill() {
         try {
           childProc.destroy();
         } catch (Exception inner) {
-          // no-op
+          try {
+            Method destroy = childProc.getClass().getMethod("destroyForcibly");
+            destroy.invoke(childProc);
+          } catch (Exception inEx){
+            // no-op.
+          }
         }
       } finally {
         childProc = null;
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 1d031073d97b..837e484cc2a6 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -126,6 +126,8 @@ public static void setConfig(String name, String value) {
   /**
    * Specifies that Spark Application be stopped if current process goes away.
    * It tries stop/kill Spark Application if {@link LauncherServer} goes away.
+   *
+   * @since 2.1.0
    * @return This launcher.
    */
   public SparkLauncher stopIfLauncherShutdown() {
@@ -137,6 +139,7 @@ public SparkLauncher stopIfLauncherShutdown() {
    * Specifies that Spark Submit be launched as a daemon thread using reflection.
    * Please note this feature is currently supported only for YARN cluster deployment mode.
    *
+   * @since 2.1.0
    * @return This launcher.
    */
   public SparkLauncher launchAsThread(boolean launchAsThread) {
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
index 77d998a58f91..f9cb14710ce2 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
@@ -58,14 +58,8 @@ public void run() {
       }
       Object argsObj = args.toArray(new String[args.size()]);
       main.invoke(null, argsObj);
-    } catch (IllegalAccessException illAcEx) {
-      throw new RuntimeException(illAcEx);
-    } catch (InvocationTargetException invokEx) {
-      throw new RuntimeException(invokEx);
-    } catch (ClassNotFoundException cnfEx) {
-      throw new RuntimeException(cnfEx);
-    } catch (NoSuchMethodException nsmEx) {
-      throw new RuntimeException(nsmEx);
+    } catch (IllegalAccessException|InvocationTargetException|ClassNotFoundException|NoSuchMethodException illEx) {
+      throw new RuntimeException(illEx);
     }
   }
 }

From ad20ccc3f4ae7eab0fb8d66b97a1d25833187f8c Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Mon, 30 Jan 2017 20:10:32 +0000
Subject: [PATCH 18/50] Fixing code to address review comments

---
 .../org/apache/spark/launcher/LauncherBackend.scala  |  2 +-
 .../apache/spark/launcher/ChildProcAppHandle.java    | 12 +++++-------
 .../org/apache/spark/launcher/SparkLauncher.java     |  4 ++--
 3 files changed, 8 insertions(+), 10 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
index 499c3380fdb6..3139456c43e9 100644
--- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
@@ -61,7 +61,7 @@ private[spark] abstract class LauncherBackend extends Logging {
         ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () =>
         logInfo("Invoking onStopRequest() from shutdown hook")
         try {
-          if (_isConnected && stopOnShutdown) {
+          if (_isConnected) {
             onStopRequest()
           }
         }
diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
index 40adb5b5b6e3..7ce5c4e4bca1 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
@@ -46,15 +46,13 @@ public synchronized void kill() {
       try {
         childProc.exitValue();
       } catch (IllegalThreadStateException e) {
+        // Child is still alive. Try to use Java 8's "destroyForcibly()" if available,
+        // fall back to the old API if it's not there.
         try {
-          childProc.destroy();
+          Method destroy = childProc.getClass().getMethod("destroyForcibly");
+          destroy.invoke(childProc);
         } catch (Exception inner) {
-          try {
-            Method destroy = childProc.getClass().getMethod("destroyForcibly");
-            destroy.invoke(childProc);
-          } catch (Exception inEx){
-            // no-op.
-          }
+          childProc.destroy();
         }
       } finally {
         childProc = null;
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 837e484cc2a6..7a010f9eecd1 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -127,7 +127,7 @@ public static void setConfig(String name, String value) {
    * Specifies that Spark Application be stopped if current process goes away.
    * It tries stop/kill Spark Application if {@link LauncherServer} goes away.
    *
-   * @since 2.1.0
+   * @since 2.2.0
    * @return This launcher.
    */
   public SparkLauncher stopIfLauncherShutdown() {
@@ -139,7 +139,7 @@ public SparkLauncher stopIfLauncherShutdown() {
    * Specifies that Spark Submit be launched as a daemon thread using reflection.
    * Please note this feature is currently supported only for YARN cluster deployment mode.
    *
-   * @since 2.1.0
+   * @since 2.2.0
    * @return This launcher.
    */
   public SparkLauncher launchAsThread(boolean launchAsThread) {

From 6a7ba5bfdd2cb165956992907f681ab3ad85154e Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 31 Jan 2017 16:07:55 +0000
Subject: [PATCH 19/50] rename variable

---
 core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 4 ++--
 .../scala/org/apache/spark/launcher/LauncherBackend.scala     | 2 +-
 .../main/java/org/apache/spark/launcher/LauncherProtocol.java | 2 +-
 .../main/java/org/apache/spark/launcher/SparkLauncher.java    | 2 +-
 4 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 473d88467594..dbec32e65ded 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -723,7 +723,7 @@ object SparkSubmit {
 
     val isSparkApp = mainClass.getMethods().filter(_.getName() == "sparkMain").length > 0
 
-    val childSparkConf = sysProps.filter( p => p._1.startsWith("spark.")).toMap
+    val childSparkConf = sysProps.filter{ p => p._1.startsWith("spark.")}.toMap
     val childSysProps = sys.env
     if (isSparkApp && threadEnabled) {
       sys.env.foreach { case (key, value) =>
@@ -738,7 +738,7 @@ object SparkSubmit {
     }
 
     val mainMethod = if (isSparkApp) {
-      mainClass.getMethods().filter(_.getName() == "sparkMain")(0)
+      mainClass.getMethods().filter{_.getName() == "sparkMain"}(0)
     } else {
       mainClass.getMethod("main", new Array[String](0).getClass)
     }
diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
index 3139456c43e9..526c9dfae9c7 100644
--- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
@@ -42,7 +42,7 @@ private[spark] abstract class LauncherBackend extends Logging {
   def connect(): Unit = {
     val port = sys.env.get(LauncherProtocol.ENV_LAUNCHER_PORT).map(_.toInt)
     val secret = sys.env.get(LauncherProtocol.ENV_LAUNCHER_SECRET)
-    val stopFlag = sys.env.get(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG).map(_.toBoolean)
+    val stopFlag = sys.env.get(LauncherProtocol.ENV_LAUNCHER_STOP_IF_SHUTDOWN).map(_.toBoolean)
     if (port != None && secret != None) {
       connect(port.get, secret.get, stopFlag.getOrElse(false))
     }
diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java
index 5081ac213812..84980b6cc3f1 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java
@@ -33,7 +33,7 @@ final class LauncherProtocol {
   static final String ENV_LAUNCHER_SECRET = "_SPARK_LAUNCHER_SECRET";
 
   /** Environment variable where the stop if Launcher is stored. */
-  static final String ENV_LAUNCHER_STOP_FLAG = "_SPARK_LAUNCHER_STOP_FLAG";
+  static final String ENV_LAUNCHER_STOP_IF_SHUTDOWN = "_SPARK_LAUNCHER_STOP_IF_SHUTDOWN";
 
   static class Message implements Serializable {
 
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 7a010f9eecd1..9e4c215d7149 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -584,7 +584,7 @@ private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] lis
     pb.environment().put(LauncherProtocol.ENV_LAUNCHER_PORT,
       String.valueOf(LauncherServer.getServerInstance().getPort()));
     pb.environment().put(LauncherProtocol.ENV_LAUNCHER_SECRET, handle.getSecret());
-    pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_FLAG, String.valueOf(stopIfLauncherShutdown));
+    pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_IF_SHUTDOWN, String.valueOf(stopIfLauncherShutdown));
     try {
       handle.setChildProc(pb.start(), loggerName);
     } catch (IOException ioe) {

From 30911050b273b38a4069a55059e9b3803e11dfa5 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 31 Jan 2017 16:54:29 +0000
Subject: [PATCH 20/50] Adding scala doc to the method

---
 core/src/main/scala/org/apache/spark/SparkApp.scala | 8 ++++++++
 1 file changed, 8 insertions(+)

diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala
index 71986f711b06..351b515ff536 100644
--- a/core/src/main/scala/org/apache/spark/SparkApp.scala
+++ b/core/src/main/scala/org/apache/spark/SparkApp.scala
@@ -24,6 +24,14 @@ package org.apache.spark
 private[spark] trait SparkApp {
   this: Singleton =>
 
+  /**
+   * The Client should implement this as entry method to provide application,
+   * spark conf and system configuration.
+   *
+   * @param args    - all argouments for SparkApp.
+   * @param conf    - Spark Configuration.
+   * @param sysConf - system configurations.
+   **/
   def sparkMain(
     args: Array[String],
     conf: scala.collection.immutable.Map[String, String],

From 2fdcec97f2455983684af8a2e6bcbe4214991972 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 31 Jan 2017 18:59:09 +0000
Subject: [PATCH 21/50] Fixing format to curly braces

---
 core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 509134f77791..4a9a3c7e16d5 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -719,7 +719,7 @@ object SparkSubmit extends CommandLineUtils {
       printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.")
     }
 
-    val isSparkApp = mainClass.getMethods().filter(_.getName() == "sparkMain").length > 0
+    val isSparkApp = mainClass.getMethods().filter{_.getName() == "sparkMain"}.length > 0
 
     val childSparkConf = sysProps.filter{ p => p._1.startsWith("spark.")}.toMap
     val childSysProps = sys.env

From 64a0e4561e3f8e11c238f1892f0a2574ff15fb9f Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Thu, 2 Feb 2017 22:31:43 +0000
Subject: [PATCH 22/50] Fixing comment formatting

---
 core/src/main/scala/org/apache/spark/SparkApp.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala
index 351b515ff536..372e71823670 100644
--- a/core/src/main/scala/org/apache/spark/SparkApp.scala
+++ b/core/src/main/scala/org/apache/spark/SparkApp.scala
@@ -31,7 +31,7 @@ private[spark] trait SparkApp {
    * @param args    - all argouments for SparkApp.
    * @param conf    - Spark Configuration.
    * @param sysConf - system configurations.
-   **/
+   */
   def sparkMain(
     args: Array[String],
     conf: scala.collection.immutable.Map[String, String],

From 43571078cd2912edb7dde07b318609b457769a39 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Mon, 6 Feb 2017 23:09:45 +0000
Subject: [PATCH 23/50] Fixing review comments on use of sys.env

---
 core/src/main/scala/org/apache/spark/SparkApp.scala   |  4 ++--
 .../scala/org/apache/spark/deploy/SparkSubmit.scala   | 11 ++---------
 .../scala/org/apache/spark/deploy/yarn/Client.scala   |  4 ++--
 3 files changed, 6 insertions(+), 13 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala
index 372e71823670..0275bf784b4d 100644
--- a/core/src/main/scala/org/apache/spark/SparkApp.scala
+++ b/core/src/main/scala/org/apache/spark/SparkApp.scala
@@ -30,11 +30,11 @@ private[spark] trait SparkApp {
    *
    * @param args    - all argouments for SparkApp.
    * @param conf    - Spark Configuration.
-   * @param sysConf - system configurations.
+   * @param envVars - system environment Variables.
    */
   def sparkMain(
     args: Array[String],
     conf: scala.collection.immutable.Map[String, String],
-    sysConf: scala.collection.immutable.Map[String, String]): Unit
+    envVars: scala.collection.immutable.Map[String, String]): Unit
 
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 4a9a3c7e16d5..4137da61bf77 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -722,14 +722,7 @@ object SparkSubmit extends CommandLineUtils {
     val isSparkApp = mainClass.getMethods().filter{_.getName() == "sparkMain"}.length > 0
 
     val childSparkConf = sysProps.filter{ p => p._1.startsWith("spark.")}.toMap
-    val childSysProps = sys.env
-    if (isSparkApp && threadEnabled) {
-      sys.env.foreach { case (key, value) =>
-        if (!sysProps.contains(key)) {
-          sysProps.put(key, value)
-        }
-      }
-    } else {
+    if (!isSparkApp || !threadEnabled) {
       sysProps.foreach { case (key, value) =>
         System.setProperty(key, value)
       }
@@ -756,7 +749,7 @@ object SparkSubmit extends CommandLineUtils {
 
     try {
       if (isSparkApp) {
-        mainMethod.invoke(null, childArgs.toArray, childSparkConf, childSysProps)
+        mainMethod.invoke(null, childArgs.toArray, childSparkConf, sys.env)
       } else {
         mainMethod.invoke(null, childArgs.toArray)
       }
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 2b2f2ebbf72a..10fc6d5ddf1e 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -1227,7 +1227,7 @@ private object Client extends SparkApp with Logging {
   override def sparkMain(
     args: Array[String],
     conf: scala.collection.immutable.Map[String, String],
-    sysConf: scala.collection.immutable.Map[String, String]): Unit = {
+    envVars: scala.collection.immutable.Map[String, String]): Unit = {
     if (!sys.props.contains("SPARK_SUBMIT")) {
       logWarning("WARNING: This client is deprecated and will be removed in a " +
         "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"")
@@ -1252,7 +1252,7 @@ private object Client extends SparkApp with Logging {
     sparkConf.remove("spark.jars")
     sparkConf.remove("spark.files")
     val argsForClient = new ClientArguments(args)
-    new Client(argsForClient, sparkConf, sysConf).run()
+    new Client(argsForClient, sparkConf, envVars).run()
   }
 
   // Alias for the user jar

From cc2c0be9aab069b827fc6d46fb8201a1902c87ba Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Fri, 10 Feb 2017 23:11:56 +0000
Subject: [PATCH 24/50] Addressing review comments on documentation and minor
 variable changes

---
 core/src/main/scala/org/apache/spark/SparkApp.scala |  6 +++---
 .../scala/org/apache/spark/deploy/SparkSubmit.scala |  7 +++++--
 .../org/apache/spark/launcher/LauncherServer.java   |  1 -
 .../org/apache/spark/launcher/SparkLauncher.java    |  2 --
 .../apache/spark/launcher/SparkSubmitRunner.java    |  2 +-
 .../org/apache/spark/launcher/package-info.java     |  7 ++++---
 .../scala/org/apache/spark/deploy/yarn/Client.scala | 13 ++++---------
 7 files changed, 17 insertions(+), 21 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala
index 0275bf784b4d..cebe8a95b688 100644
--- a/core/src/main/scala/org/apache/spark/SparkApp.scala
+++ b/core/src/main/scala/org/apache/spark/SparkApp.scala
@@ -28,13 +28,13 @@ private[spark] trait SparkApp {
    * The Client should implement this as entry method to provide application,
    * spark conf and system configuration.
    *
-   * @param args    - all argouments for SparkApp.
+   * @param args    - all arguments for SparkApp.
    * @param conf    - Spark Configuration.
-   * @param envVars - system environment Variables.
+   * @param envvars - system environment Variables.
    */
   def sparkMain(
     args: Array[String],
     conf: scala.collection.immutable.Map[String, String],
-    envVars: scala.collection.immutable.Map[String, String]): Unit
+    envvars: scala.collection.immutable.Map[String, String]): Unit
 
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 4137da61bf77..c382442b15db 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -719,9 +719,12 @@ object SparkSubmit extends CommandLineUtils {
       printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.")
     }
 
-    val isSparkApp = mainClass.getMethods().filter{_.getName() == "sparkMain"}.length > 0
+    val sparkAppMainMethodArr = mainClass.getMethods().filter{_.getName() == "sparkMain"}
+    val isSparkApp = sparkAppMainMethodArr.length > 0
 
     val childSparkConf = sysProps.filter{ p => p._1.startsWith("spark.")}.toMap
+    // If running sparkApp or in thread mode, the System properties should not be cluttered.
+    // This helps keep clean isolation between multiple Spark Apps launched in different threads.
     if (!isSparkApp || !threadEnabled) {
       sysProps.foreach { case (key, value) =>
         System.setProperty(key, value)
@@ -729,7 +732,7 @@ object SparkSubmit extends CommandLineUtils {
     }
 
     val mainMethod = if (isSparkApp) {
-      mainClass.getMethods().filter{_.getName() == "sparkMain"}(0)
+      sparkAppMainMethodArr(0)
     } else {
       mainClass.getMethod("main", new Array[String](0).getClass)
     }
diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
index 2d5c9a9ee4e8..4e1c87efcc04 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
@@ -108,7 +108,6 @@ static synchronized ChildThreadAppHandle newAppThreadHandle() throws IOException
     return (ChildThreadAppHandle) LauncherServer.newAppHandle(true);
   }
 
-
   /**
    * Creates a handle for an app to be launched. This method will start a server if one hasn't been
    * started yet. The server is shared for multiple handles, and once all handles are disposed of,
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 9e4c215d7149..40d0540accc5 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -103,13 +103,11 @@ public class SparkLauncher {
 
   static final Map launcherConfig = new HashMap<>();
 
-
   private boolean stopIfLauncherShutdown = false;
 
   /** Flag to decide on launching spark-submit as a child process or a thread **/
   private boolean launchAsThread = false;
 
-
   /**
    * Set a configuration value for the launcher library. These config values do not affect the
    * launched application, but rather the behavior of the launcher library itself when managing
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
index f9cb14710ce2..cd5b4761b2c5 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
@@ -25,7 +25,7 @@
  * This class used to by {@link SparkLauncher#startApplicationAsThread(SparkAppHandle.Listener...)}
  * 

* This class to start Spark applications programmatically as a Thread. This uses reflection and relies - * on find SparkSubmit main class. + * on the SparkSubmit main class being in the classpath. *

*/ class SparkSubmitRunner implements Runnable { diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java index dd53e61a6558..620903a399ec 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -25,8 +25,8 @@ * *

* The {@link org.apache.spark.launcher.SparkLauncher#startApplication( - * org.apache.spark.launcher.SparkAppHandle.Listener...)} can be used to start Spark and provide - * a handle to monitor and control the running application: + * org.apache.spark.launcher.SparkAppHandle.Listener...)} can be used to start Spark application + * as a process and providea handle to monitor and control the running application: *

* *
@@ -49,7 +49,7 @@
  * 
* *

- * Currently, for currently while launching spark application with + * Currently, while launching spark application with * {@link org.apache.spark.launcher.SparkLauncher#startApplication( * org.apache.spark.launcher.SparkAppHandle.Listener...)}, there are two options available * for YARN manager in cluster deploy mode: @@ -70,6 +70,7 @@ * .setAppResource("/my/app.jar") * .setMainClass("my.spark.app.Main") * .setMaster("yarn") + * .setDeployMode("cluster") * .stopIfLauncherShutdown() * .launchAsThread(true) * .setConf(SparkLauncher.DRIVER_MEMORY, "2g") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index c4761c77922a..1cef4742cfe8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1170,7 +1170,7 @@ private object Client extends SparkApp with Logging { override def sparkMain( args: Array[String], conf: scala.collection.immutable.Map[String, String], - envVars: scala.collection.immutable.Map[String, String]): Unit = { + envvars: scala.collection.immutable.Map[String, String]): Unit = { if (!sys.props.contains("SPARK_SUBMIT")) { logWarning("WARNING: This client is deprecated and will be removed in a " + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") @@ -1180,14 +1180,9 @@ private object Client extends SparkApp with Logging { // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val threadEnabled: Boolean = conf.get(SparkLauncher.LAUNCHER_INTERNAL_THREAD_ENABLED). - getOrElse("false").toBoolean - val sparkConf = new SparkConf - if (threadEnabled) { - for ((key, value) <- conf if key.startsWith("spark.")) { - sparkConf.set(key, value, true) - } + for ((key, value) <- conf if key.startsWith("spark.")) { + sparkConf.set(key, value, true) } // SparkSubmit would use yarn cache to distribute files & jars in yarn mode, @@ -1195,7 +1190,7 @@ private object Client extends SparkApp with Logging { sparkConf.remove("spark.jars") sparkConf.remove("spark.files") val argsForClient = new ClientArguments(args) - new Client(argsForClient, sparkConf, envVars).run() + new Client(argsForClient, sparkConf, envvars).run() } // Alias for the user jar From 82df055699b6d4010108e1bd41cb276e65657a00 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Mon, 6 Mar 2017 19:24:31 +0000 Subject: [PATCH 25/50] Addressing review comments --- .../scala/org/apache/spark/SparkApp.scala | 4 +-- .../org/apache/spark/deploy/SparkSubmit.scala | 13 +++---- .../apache/spark/examples/JavaWordCount.java | 8 ++--- .../spark/launcher/ChildThreadAppHandle.java | 2 +- .../spark/launcher/LauncherConnection.java | 1 + .../apache/spark/launcher/LauncherServer.java | 7 ++-- .../apache/spark/launcher/SparkLauncher.java | 35 ++++++++++--------- .../spark/launcher/SparkSubmitRunner.java | 1 - .../apache/spark/launcher/package-info.java | 18 +++++----- .../org/apache/spark/deploy/yarn/Client.scala | 2 +- 10 files changed, 43 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala index cebe8a95b688..97298197c271 100644 --- a/core/src/main/scala/org/apache/spark/SparkApp.scala +++ b/core/src/main/scala/org/apache/spark/SparkApp.scala @@ -34,7 +34,7 @@ private[spark] trait SparkApp { */ def sparkMain( args: Array[String], - conf: scala.collection.immutable.Map[String, String], - envvars: scala.collection.immutable.Map[String, String]): Unit + conf: Map[String, String], + envvars: Map[String, String]): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c382442b15db..ad2a4b545c72 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -686,9 +686,6 @@ object SparkSubmit extends CommandLineUtils { addJarToClasspath(jar, loader) } - val threadEnabled = sysProps.getOrElse(SparkLauncher.LAUNCHER_INTERNAL_THREAD_ENABLED, - "false").toBoolean - var mainClass: Class[_] = null try { @@ -719,13 +716,17 @@ object SparkSubmit extends CommandLineUtils { printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.") } - val sparkAppMainMethodArr = mainClass.getMethods().filter{_.getName() == "sparkMain"} + val sparkAppMainMethodArr = mainClass.getMethods().filter(_.getName() == "sparkMain") val isSparkApp = sparkAppMainMethodArr.length > 0 - val childSparkConf = sysProps.filter{ p => p._1.startsWith("spark.")}.toMap + val childSparkConf = sysProps.filter( p => p._1.startsWith("spark.")).toMap + + val threadEnabled = sysProps.getOrElse(SparkLauncher.LAUNCHER_INTERNAL_USE_THREAD, + "false").toBoolean + // If running sparkApp or in thread mode, the System properties should not be cluttered. // This helps keep clean isolation between multiple Spark Apps launched in different threads. - if (!isSparkApp || !threadEnabled) { + if (!isSparkApp) { sysProps.foreach { case (key, value) => System.setProperty(key, value) } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 8f18604c0750..5e60b0c7bcec 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -36,17 +36,13 @@ public final class JavaWordCount { public static void main(String[] args) throws Exception { - if (args.length < 1) { - System.err.println("Usage: JavaWordCount "); - System.exit(1); - } - + String fileName= "READTHIS.md"; SparkSession spark = SparkSession .builder() .appName("JavaWordCount") .getOrCreate(); - JavaRDD lines = spark.read().textFile(args[0]).javaRDD(); + JavaRDD lines = spark.read().textFile(fileName).javaRDD(); JavaRDD words = lines.flatMap(new FlatMapFunction() { @Override diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java index eafdb5411d73..23185d10cca6 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java @@ -51,4 +51,4 @@ public synchronized void kill() { void setChildThread(Thread childThread) { this.childThread = childThread; } -} \ No newline at end of file +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java index 5991ec80a2bd..eec264909bbb 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java @@ -106,4 +106,5 @@ public void close() throws IOException { } } } + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index 4e1c87efcc04..be2a2ba492f3 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -90,9 +90,9 @@ class LauncherServer implements Closeable { /** * Creates a handle for an app to be launched using ChildProc. - * This method will start a server if one hasn't been - * started yet. The server is shared for multiple handles, and once all handles are disposed of, - * the server is shut down. + * This method will start a server if one hasn't been started yet. + * The server is shared for multiple handles, and once all handles + * are disposed of, the server is shut down. */ static synchronized ChildProcAppHandle newAppHandle() throws IOException { return (ChildProcAppHandle) LauncherServer.newAppHandle(false); @@ -177,7 +177,6 @@ public void run() { * amount of time, and if one doesn't arrive, it will transition to an error state. */ AbstractSparkAppHandle newAppHandle(String secret, boolean isThreadHandle) { - AbstractSparkAppHandle handle; if (isThreadHandle) { handle = new ChildThreadAppHandle(secret, this); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 40d0540accc5..15bf80cf30c7 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -78,9 +78,9 @@ public class SparkLauncher { public static final String LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET = "spark.launcher.internal.secret"; - public static final String LAUNCHER_INTERNAL_STOP_IF_SHUTDOWN = "spark.launcher.internal.stop.if.shutdown"; + public static final String LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN = "spark.launcher.internal.stop.on.shutdown"; - public static final String LAUNCHER_INTERNAL_THREAD_ENABLED = "spark.launcher.internal.thread.enabled"; + public static final String LAUNCHER_INTERNAL_USE_THREAD = "spark.launcher.internal.use.thread"; /** * A special value for the resource that tells Spark to not try to process the app resource as a @@ -103,7 +103,7 @@ public class SparkLauncher { static final Map launcherConfig = new HashMap<>(); - private boolean stopIfLauncherShutdown = false; + private boolean stopOnShutdown = false; /** Flag to decide on launching spark-submit as a child process or a thread **/ private boolean launchAsThread = false; @@ -128,14 +128,14 @@ public static void setConfig(String name, String value) { * @since 2.2.0 * @return This launcher. */ - public SparkLauncher stopIfLauncherShutdown() { - this.stopIfLauncherShutdown = true; + public SparkLauncher autoShutdown() { + this.stopOnShutdown = true; return this; } /** - * Specifies that Spark Submit be launched as a daemon thread using reflection. - * Please note this feature is currently supported only for YARN cluster deployment mode. + * Specifies that Spark Submit be launched as a daemon thread. Please note + * this feature is currently supported only for YARN cluster deployment mode. * * @since 2.2.0 * @return This launcher. @@ -538,7 +538,7 @@ private String getAppName() throws IOException { * that happens after that cannot be monitored. If the underlying application is launched as * a child process, {@link SparkAppHandle#kill()} can still be used to kill the child process. *

- * If the applications are launched as child processes. The child's stdout and stderr + * If the application is launched as child process, the child's stdout and stderr * are merged and written to a logger (see java.util.logging) only if redirection * has not otherwise been configured on this SparkLauncher. The logger's name can be * defined by setting {@link #CHILD_PROCESS_LOGGER_NAME} in the app's configuration. If that @@ -547,9 +547,8 @@ private String getAppName() throws IOException { * In all cases, the logger name will start with "org.apache.spark.launcher.app", to fit more * easily into the configuration of commonly-used logging systems. * - * If the applications are launched as a thread, the {@link SparkLauncher#redirectError} and - * {@link SparkLauncher#redirectToLog}, are not supported at this time. The existing process - * stdout and stderr will get all the log entries. + * If the application is launched as a thread, the log redirection methods are not supported, + * and the parent process's output and log configuration will be used. * * @since 1.6.0 * @param listeners Listeners to add to the handle before the app is launched. @@ -582,7 +581,7 @@ private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] lis pb.environment().put(LauncherProtocol.ENV_LAUNCHER_PORT, String.valueOf(LauncherServer.getServerInstance().getPort())); pb.environment().put(LauncherProtocol.ENV_LAUNCHER_SECRET, handle.getSecret()); - pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_IF_SHUTDOWN, String.valueOf(stopIfLauncherShutdown)); + pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_IF_SHUTDOWN, String.valueOf(stopOnShutdown)); try { handle.setChildProc(pb.start(), loggerName); } catch (IOException ioe) { @@ -601,10 +600,10 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste String appName = getAppName(); setConf(LAUNCHER_INTERNAL_PORT, String.valueOf(LauncherServer.getServerInstance().getPort())); setConf(LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, handle.getSecret()); - setConf(LAUNCHER_INTERNAL_STOP_IF_SHUTDOWN, String.valueOf(stopIfLauncherShutdown)); - setConf(LAUNCHER_INTERNAL_THREAD_ENABLED, "true"); + setConf(LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, String.valueOf(stopOnShutdown)); + setConf(LAUNCHER_INTERNAL_USE_THREAD, "true"); try { - // It is important that spark-submit class is available in the classpath. + // It is important that SparkSubmit class is available in the classpath. // Trying to see if method is available in the classpath else throws Exception. Method main = SparkSubmitRunner.getSparkSubmitMain(); Thread submitJobThread = new Thread(new SparkSubmitRunner(main, builder.buildSparkSubmitArgs())); @@ -613,9 +612,11 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste handle.setChildThread(submitJobThread); submitJobThread.start(); } catch (ClassNotFoundException cnfe) { - throw new IOException("Please make sure spark-core is in the classpath.", cnfe); + throw new IOException("Please make sure the spark jar containing SparkSubmit is in the classpath.", + cnfe); } catch (NoSuchMethodException nsme) { - throw new IOException("Please make sure spark-core version is correct.", nsme); + throw new IOException("Please make sure the spark jar containing SparkSubmit version is correct.", + nsme); } return handle; } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java index cd5b4761b2c5..f2af68cf3d8d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java @@ -38,7 +38,6 @@ class SparkSubmitRunner implements Runnable { } /** - * It is important that spark-submit class is available in the classpath. * Trying to see if "org.apache.spark.deploy.SparkSubmit#Main" method is available in the classpath. * * @return Method to execute for starting Spark Application diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java index 620903a399ec..9b10d2b08884 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -21,12 +21,14 @@ *

* This library allows applications to launch Spark programmatically. There's only one entry * point to the library - the {@link org.apache.spark.launcher.SparkLauncher} class. + * Under YARN manager cluster mode, it supports launching in Application in thread or + * child JVM process. *

* *

* The {@link org.apache.spark.launcher.SparkLauncher#startApplication( * org.apache.spark.launcher.SparkAppHandle.Listener...)} can be used to start Spark application - * as a process and providea handle to monitor and control the running application: + * as a process and provide a handle to monitor and control the running application: *

* *
@@ -49,14 +51,10 @@
  * 
* *

- * Currently, while launching spark application with - * {@link org.apache.spark.launcher.SparkLauncher#startApplication( - * org.apache.spark.launcher.SparkAppHandle.Listener...)}, there are two options available - * for YARN manager in cluster deploy mode: - * - to launch Spark Application as a Thread inside current JVM using - * the {@link org.apache.spark.launcher.SparkLauncher#launchAsThread(boolean)} - * - to request application be killed if launcher process exits using - * the {@link org.apache.spark.launcher.SparkLauncher#stopIfLauncherShutdown()}. + * Here is example of launching application in thread mode for yarn cluster mode + * with the stop if launcher shutdown option enabled. Showing use case of both + * {@link org.apache.spark.launcher.SparkLauncher#stopOnShutdown} and + * {@link org.apache.spark.launcher.SparkLauncher#launchAsThread} methods *

* *
@@ -71,7 +69,7 @@
  *         .setMainClass("my.spark.app.Main")
  *         .setMaster("yarn")
  *         .setDeployMode("cluster")
- *         .stopIfLauncherShutdown()
+ *         .autoShutdown()
  *         .launchAsThread(true)
  *         .setConf(SparkLauncher.DRIVER_MEMORY, "2g")
  *         .startApplication();
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 1cef4742cfe8..76165b0afd26 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -88,7 +88,7 @@ private[spark] class Client(
   private val launcherServerSecret : String =
     sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, "")
   private val launcherServerStopIfShutdown : Boolean =
-    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_STOP_IF_SHUTDOWN, "false").toBoolean
+    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, "false").toBoolean
   // AM related configurations
   private val amMemory = if (isClusterMode) {
     sparkConf.get(DRIVER_MEMORY).toInt

From 99d8c29d2c337e84588e8b4951c404565838153b Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 7 Mar 2017 20:29:12 +0000
Subject: [PATCH 26/50] Fixing few more comments

---
 .../scala/org/apache/spark/deploy/SparkSubmit.scala   |  6 ++----
 .../org/apache/spark/launcher/LauncherBackend.scala   | 11 ++---------
 .../java/org/apache/spark/examples/JavaWordCount.java |  8 ++++++--
 .../apache/spark/launcher/AbstractSparkAppHandle.java |  2 +-
 .../java/org/apache/spark/launcher/SparkLauncher.java |  3 ---
 .../apache/spark/deploy/yarn/ApplicationMaster.scala  |  2 +-
 6 files changed, 12 insertions(+), 20 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index ad2a4b545c72..3d1336253a28 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -721,9 +721,6 @@ object SparkSubmit extends CommandLineUtils {
 
     val childSparkConf = sysProps.filter( p => p._1.startsWith("spark.")).toMap
 
-    val threadEnabled = sysProps.getOrElse(SparkLauncher.LAUNCHER_INTERNAL_USE_THREAD,
-      "false").toBoolean
-
     // If running sparkApp or in thread mode, the System properties should not be cluttered.
     // This helps keep clean isolation between multiple Spark Apps launched in different threads.
     if (!isSparkApp) {
@@ -753,7 +750,8 @@ object SparkSubmit extends CommandLineUtils {
 
     try {
       if (isSparkApp) {
-        mainMethod.invoke(null, childArgs.toArray, childSparkConf, sys.env)
+        val envvars = Map[String, String]() ++ sys.env
+        mainMethod.invoke(null, childArgs.toArray, childSparkConf, envvars.toMap)
       } else {
         mainMethod.invoke(null, childArgs.toArray)
       }
diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
index 526c9dfae9c7..8c164cba3d14 100644
--- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
@@ -48,7 +48,8 @@ private[spark] abstract class LauncherBackend extends Logging {
     }
   }
 
-  def connect(port: Int, secret: String): Unit = {
+  def connect(port: Int, secret: String, stopFlag: Boolean): Unit = {
+    this.stopOnShutdown = stopFlag
     val s = new Socket(InetAddress.getLoopbackAddress(), port)
     connection = new BackendConnection(s)
     connection.send(new Hello(secret, SPARK_VERSION))
@@ -73,11 +74,6 @@ private[spark] abstract class LauncherBackend extends Logging {
     }
   }
 
-  def connect(port: Int, secret: String, stopFlag: Boolean): Unit = {
-    this.stopOnShutdown = stopFlag
-    connect(port, secret)
-  }
-
   def close(): Unit = {
     if (connection != null) {
       try {
@@ -100,9 +96,6 @@ private[spark] abstract class LauncherBackend extends Logging {
     if (connection != null && lastState != state) {
       connection.send(new SetState(state))
       lastState = state
-      if (!_isConnected && stopOnShutdown) {
-        fireStopRequest()
-      }
     }
   }
 
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
index 5e60b0c7bcec..8f18604c0750 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
@@ -36,13 +36,17 @@ public final class JavaWordCount {
 
   public static void main(String[] args) throws Exception {
 
-    String fileName= "READTHIS.md";
+    if (args.length < 1) {
+      System.err.println("Usage: JavaWordCount ");
+      System.exit(1);
+    }
+
     SparkSession spark = SparkSession
       .builder()
       .appName("JavaWordCount")
       .getOrCreate();
 
-    JavaRDD lines = spark.read().textFile(fileName).javaRDD();
+    JavaRDD lines = spark.read().textFile(args[0]).javaRDD();
 
     JavaRDD words = lines.flatMap(new FlatMapFunction() {
       @Override
diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java
index 97287b34d2e2..36d2557de841 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java
@@ -23,7 +23,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-public abstract class AbstractSparkAppHandle implements SparkAppHandle {
+abstract class AbstractSparkAppHandle implements SparkAppHandle {
   private static final Logger LOG = Logger.getLogger(AbstractSparkAppHandle.class.getName());
 
   protected final String secret;
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 15bf80cf30c7..d92c5216e56f 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -80,8 +80,6 @@ public class SparkLauncher {
 
   public static final String LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN = "spark.launcher.internal.stop.on.shutdown";
 
-  public static final String LAUNCHER_INTERNAL_USE_THREAD = "spark.launcher.internal.use.thread";
-
   /**
    * A special value for the resource that tells Spark to not try to process the app resource as a
    * file. This is useful when the class being executed is added to the application using other
@@ -601,7 +599,6 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste
     setConf(LAUNCHER_INTERNAL_PORT, String.valueOf(LauncherServer.getServerInstance().getPort()));
     setConf(LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, handle.getSecret());
     setConf(LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, String.valueOf(stopOnShutdown));
-    setConf(LAUNCHER_INTERNAL_USE_THREAD, "true");
     try {
       // It is important that SparkSubmit class is available in the classpath.
       // Trying to see if method is available in the classpath else throws Exception.
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index b47f90efa2ab..12f8bd74eccc 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -175,7 +175,7 @@ private[spark] class ApplicationMaster(
     }
 
     // Clean up the spark launcher configuration so it doesn't show up in the Web UI for security.
-    Seq("spark.launcher.internal.secret", "spark.launcher.internal.port").foreach { e =>
+    LAUNCHER_CONFIGs.foreach { e =>
       sparkConf.remove(e)
       sys.props.remove(e)
     }

From 41cf6daad85f6957017fad5c2946fea681f81361 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Wed, 8 Mar 2017 06:25:38 +0000
Subject: [PATCH 27/50] Adding tests for SparkLauncher in thread model

---
 .../spark/launcher/SparkLauncherSuite.java    | 31 +++++++++
 .../spark/deploy/yarn/YarnClusterSuite.scala  | 68 +++++++++++++++++++
 2 files changed, 99 insertions(+)

diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
index 0c7712374085..6b1f1c7d589e 100644
--- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
+++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -175,6 +175,37 @@ public void testChildProcLauncher() throws Exception {
       .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path"))
       .addSparkArg(opts.CLASS, "ShouldBeOverriddenBelow")
       .setMainClass(SparkLauncherTestApp.class.getName())
+      .autoShutdown()
+      .addAppArgs("proc");
+    final Process app = launcher.launch();
+
+    new OutputRedirector(app.getInputStream(), TF);
+    new OutputRedirector(app.getErrorStream(), TF);
+    assertEquals(0, app.waitFor());
+  }
+
+  @Test
+  public void testThreadLauncher() throws Exception {
+    // This test is failed on Windows due to the failure of initiating executors
+    // by the path length limitation. See SPARK-18718.
+    assumeTrue(!Utils.isWindows());
+
+    SparkSubmitOptionParser opts = new SparkSubmitOptionParser();
+    Map env = new HashMap<>();
+    env.put("SPARK_PRINT_LAUNCH_COMMAND", "1");
+
+    launcher
+      .setMaster("local")
+      .setAppResource(SparkLauncher.NO_RESOURCE)
+      .addSparkArg(opts.CONF,
+        String.format("%s=-Dfoo=ShouldBeOverriddenBelow", SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS))
+      .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS,
+        "-Dfoo=bar -Dtest.appender=childproc")
+      .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path"))
+      .addSparkArg(opts.CLASS, "ShouldBeOverriddenBelow")
+      .setMainClass(SparkLauncherTestApp.class.getName())
+      .launchAsThread(true)
+      .autoShutdown()
       .addAppArgs("proc");
     final Process app = launcher.launch();
 
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 99fb58a28934..4bec4d264a4e 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -193,6 +193,74 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     }
   }
 
+  test("monitor app using launcher library for thread") {
+    val env = new JHashMap[String, String]()
+    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
+
+    val propsFile = createConfFile()
+    val handle = new SparkLauncher(env)
+      .setSparkHome(sys.props("spark.test.home"))
+      .setConf("spark.ui.enabled", "false")
+      .setPropertiesFile(propsFile)
+      .setMaster("yarn")
+      .setDeployMode("cluster")
+      .launchAsThread(true)
+      .setAppResource(SparkLauncher.NO_RESOURCE)
+      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
+      .startApplication()
+
+    try {
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.RUNNING)
+      }
+
+      handle.getAppId() should not be (null)
+      handle.getAppId() should startWith ("application_")
+      handle.stop()
+
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.KILLED)
+      }
+    } finally {
+      handle.kill()
+    }
+  }
+
+  test("monitor app using launcher library for thread with auto shutdown") {
+    val env = new JHashMap[String, String]()
+    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
+
+    val propsFile = createConfFile()
+    val handle = new SparkLauncher(env)
+      .setSparkHome(sys.props("spark.test.home"))
+      .setConf("spark.ui.enabled", "false")
+      .setPropertiesFile(propsFile)
+      .setMaster("yarn")
+      .setDeployMode("cluster")
+      .launchAsThread(true)
+      .autoShutdown()
+      .setAppResource(SparkLauncher.NO_RESOURCE)
+      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
+      .startApplication()
+
+    try {
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.RUNNING)
+      }
+
+      handle.getAppId() should not be (null)
+      handle.getAppId() should startWith ("application_")
+      handle.disconnect()
+
+
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.LOST)
+      }
+    } finally {
+      handle.kill()
+    }
+  }
+
   test("timeout to get SparkContext in cluster mode triggers failure") {
     val timeout = 2000
     val finalState = runSpark(false, mainClassName(SparkContextTimeoutApp.getClass),

From b098ecd6e21b60ae3db13adc81d02522c2fa9391 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Wed, 8 Mar 2017 06:39:20 +0000
Subject: [PATCH 28/50] Update documenation

---
 .../src/main/java/org/apache/spark/launcher/SparkAppHandle.java | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
index 0aa7bd197d16..29b48dacd421 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
@@ -100,6 +100,8 @@ public boolean isFinal() {
   /**
    * Disconnects the handle from the application, without stopping it. After this method is called,
    * the handle will not be able to communicate with the application anymore.
+   * Please note if using {@link SparkLauncher#autoShutdown()} option, this method would shutdown
+   * stop/kill the application.
    */
   void disconnect();
 

From b66243dd0644b6c80cece4a9995f9fe256d6cb21 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Fri, 10 Mar 2017 08:02:11 +0000
Subject: [PATCH 29/50] Adding code review comments on documentation

---
 .../src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 4 ++--
 .../main/java/org/apache/spark/launcher/SparkAppHandle.java  | 5 ++---
 .../org/apache/spark/deploy/yarn/ApplicationMaster.scala     | 2 +-
 .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +-
 .../src/main/scala/org/apache/spark/deploy/yarn/config.scala | 2 +-
 .../org/apache/spark/deploy/yarn/YarnClusterSuite.scala      | 1 -
 6 files changed, 7 insertions(+), 9 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 3d1336253a28..ca95f79a31b9 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -721,8 +721,8 @@ object SparkSubmit extends CommandLineUtils {
 
     val childSparkConf = sysProps.filter( p => p._1.startsWith("spark.")).toMap
 
-    // If running sparkApp or in thread mode, the System properties should not be cluttered.
-    // This helps keep clean isolation between multiple Spark Apps launched in different threads.
+    // If running a SparkApp we can explicitly pass in the confs separately.
+    // If we aren't running a SparkApp they get passed via the system properties.
     if (!isSparkApp) {
       sysProps.foreach { case (key, value) =>
         System.setProperty(key, value)
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
index 29b48dacd421..917fff3eaa1a 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
@@ -98,10 +98,9 @@ public boolean isFinal() {
   void kill();
 
   /**
-   * Disconnects the handle from the application, without stopping it. After this method is called,
+   * Disconnects the handle from the application. If using {@link SparkLauncher#autoShutdown()}
+   * option, this method would shutdown stop/kill the application. After this method is called,
    * the handle will not be able to communicate with the application anymore.
-   * Please note if using {@link SparkLauncher#autoShutdown()} option, this method would shutdown
-   * stop/kill the application.
    */
   void disconnect();
 
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 12f8bd74eccc..f851c4dcfc9b 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -175,7 +175,7 @@ private[spark] class ApplicationMaster(
     }
 
     // Clean up the spark launcher configuration so it doesn't show up in the Web UI for security.
-    LAUNCHER_CONFIGs.foreach { e =>
+    LAUNCHER_CONFIGS.foreach { e =>
       sparkConf.remove(e)
       sys.props.remove(e)
     }
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 76165b0afd26..795af7400850 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -636,7 +636,7 @@ private[spark] class Client(
     // polluting the UI's environment page. This works for client mode; for cluster mode, this
     // is handled by the AM.
     CACHE_CONFIGS.foreach(sparkConf.remove)
-    LAUNCHER_CONFIGs.foreach(sparkConf.remove)
+    LAUNCHER_CONFIGS.foreach(sparkConf.remove)
 
     localResources
   }
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
index 3f86af58ec2b..2e81b551de67 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -350,7 +350,7 @@ package object config {
     CACHED_FILES_TYPES,
     CACHED_CONF_ARCHIVE)
 
-  private[yarn] val LAUNCHER_CONFIGs = Seq(
+  private[yarn] val LAUNCHER_CONFIGS = Seq(
     SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET,
     SparkLauncher.LAUNCHER_INTERNAL_PORT)
 
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 4bec4d264a4e..451ec3f284a5 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -252,7 +252,6 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       handle.getAppId() should startWith ("application_")
       handle.disconnect()
 
-
       eventually(timeout(30 seconds), interval(100 millis)) {
         handle.getState() should be (SparkAppHandle.State.LOST)
       }

From bc994356987a2e6e321a0f8f23ffad0797de22d5 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 14 Mar 2017 02:02:02 -0400
Subject: [PATCH 30/50] Adding integration tests

---
 .../deploy/yarn/BaseYarnClusterSuite.scala    |   3 +-
 .../spark/deploy/yarn/YarnClusterSuite.scala  | 108 ++++++++++++++++--
 2 files changed, 100 insertions(+), 11 deletions(-)

diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
index 9c3b18e4ec5f..c43aa92fd12a 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
@@ -56,7 +56,7 @@ abstract class BaseYarnClusterSuite
     |log4j.logger.org.spark_project.jetty=WARN
     """.stripMargin
 
-  private var yarnCluster: MiniYARNCluster = _
+  protected var yarnCluster: MiniYARNCluster = _
   protected var tempDir: File = _
   private var fakeSparkJar: File = _
   protected var hadoopConfDir: File = _
@@ -66,6 +66,7 @@ abstract class BaseYarnClusterSuite
 
   def newYarnConfig(): YarnConfiguration
 
+
   override def beforeAll() {
     super.beforeAll()
     oldSystemProperties = SerializationUtils.clone(System.getProperties)
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 451ec3f284a5..6e63040bed49 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -25,19 +25,19 @@ import java.util.{HashMap => JHashMap}
 import scala.collection.mutable
 import scala.concurrent.duration._
 import scala.language.postfixOps
-
 import com.google.common.io.{ByteStreams, Files}
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.scalatest.Matchers
 import org.scalatest.concurrent.Eventually._
-
 import org.apache.spark._
 import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.hadoop.yarn.client.api.YarnClient
+import org.apache.hadoop.yarn.api.records.{FinalApplicationStatus, YarnApplicationState}
+import org.apache.hadoop.yarn.util.ConverterUtils
 import org.apache.spark.deploy.yarn.config._
 import org.apache.spark.internal.Logging
 import org.apache.spark.launcher._
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart,
-  SparkListenerExecutorAdded}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded}
 import org.apache.spark.scheduler.cluster.ExecutorInfo
 import org.apache.spark.tags.ExtendedYarnTest
 import org.apache.spark.util.Utils
@@ -79,6 +79,13 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     |    return 42
     """.stripMargin
 
+  private def getYarnClient = {
+    var yarnClient = YarnClient.createYarnClient()
+    yarnClient.init(yarnCluster.getConfig)
+    yarnClient.start()
+    yarnClient
+  }
+
   test("run Spark in yarn-client mode") {
     testBasicYarnApp(true)
   }
@@ -193,6 +200,14 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     }
   }
 
+  test("timeout to get SparkContext in cluster mode triggers failure") {
+    val timeout = 2000
+    val finalState = runSpark(false, mainClassName(SparkContextTimeoutApp.getClass),
+      appArgs = Seq((timeout * 4).toString),
+      extraConf = Map(AM_MAX_WAIT_TIME.key -> timeout.toString))
+    finalState should be (SparkAppHandle.State.FAILED)
+  }
+
   test("monitor app using launcher library for thread") {
     val env = new JHashMap[String, String]()
     env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
@@ -226,6 +241,44 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     }
   }
 
+  test("monitor app using launcher library for proc with auto shutdown") {
+    val env = new JHashMap[String, String]()
+    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
+
+    val propsFile = createConfFile()
+    val handle = new SparkLauncher(env)
+      .setSparkHome(sys.props("spark.test.home"))
+      .setConf("spark.ui.enabled", "false")
+      .setPropertiesFile(propsFile)
+      .setMaster("yarn")
+      .setDeployMode("cluster")
+      .autoShutdown()
+      .setAppResource(SparkLauncher.NO_RESOURCE)
+      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
+      .startApplication()
+
+    try {
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.RUNNING)
+      }
+
+      handle.getAppId() should not be (null)
+      handle.getAppId() should startWith ("application_")
+      handle.disconnect()
+
+      var applicationId = ConverterUtils.toApplicationId(handle.getAppId)
+      var yarnClient: YarnClient = getYarnClient
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.LOST)
+        var status = yarnClient.getApplicationReport(applicationId).getFinalApplicationStatus()
+        status should be (FinalApplicationStatus.KILLED)
+      }
+
+    } finally {
+      handle.kill()
+    }
+  }
+
   test("monitor app using launcher library for thread with auto shutdown") {
     val env = new JHashMap[String, String]()
     env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
@@ -252,20 +305,55 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       handle.getAppId() should startWith ("application_")
       handle.disconnect()
 
+      var applicationId = ConverterUtils.toApplicationId(handle.getAppId)
+      var yarnClient: YarnClient = getYarnClient
       eventually(timeout(30 seconds), interval(100 millis)) {
         handle.getState() should be (SparkAppHandle.State.LOST)
+        var status = yarnClient.getApplicationReport(applicationId).getFinalApplicationStatus
+        status should be (FinalApplicationStatus.KILLED)
       }
+
     } finally {
       handle.kill()
     }
   }
 
-  test("timeout to get SparkContext in cluster mode triggers failure") {
-    val timeout = 2000
-    val finalState = runSpark(false, mainClassName(SparkContextTimeoutApp.getClass),
-      appArgs = Seq((timeout * 4).toString),
-      extraConf = Map(AM_MAX_WAIT_TIME.key -> timeout.toString))
-    finalState should be (SparkAppHandle.State.FAILED)
+  test("monitor app using launcher library for thread without auto shutdown") {
+    val env = new JHashMap[String, String]()
+    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
+
+    val propsFile = createConfFile()
+    val handle = new SparkLauncher(env)
+      .setSparkHome(sys.props("spark.test.home"))
+      .setConf("spark.ui.enabled", "false")
+      .setPropertiesFile(propsFile)
+      .setMaster("yarn")
+      .setDeployMode("cluster")
+      .launchAsThread(true)
+      .setAppResource(SparkLauncher.NO_RESOURCE)
+      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
+      .startApplication()
+
+    try {
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.RUNNING)
+      }
+
+      handle.getAppId() should not be (null)
+      handle.getAppId() should startWith ("application_")
+      handle.disconnect()
+
+      var applicationId = ConverterUtils.toApplicationId(handle.getAppId)
+      var yarnClient: YarnClient = getYarnClient
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.LOST)
+        var status = yarnClient.getApplicationReport(applicationId).getYarnApplicationState
+        status should not be (FinalApplicationStatus.KILLED)
+      }
+
+    } finally {
+      handle.kill()
+    }
   }
 
   private def testBasicYarnApp(clientMode: Boolean, conf: Map[String, String] = Map()): Unit = {

From 517fed0f0e3b32cb0532bf53e9fc16d5006efce5 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 14 Mar 2017 03:37:41 -0400
Subject: [PATCH 31/50] Fixing scala style

---
 .../apache/spark/deploy/yarn/YarnClusterSuite.scala    | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)

diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 6e63040bed49..4187bf7f312d 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -25,15 +25,17 @@ import java.util.{HashMap => JHashMap}
 import scala.collection.mutable
 import scala.concurrent.duration._
 import scala.language.postfixOps
+
 import com.google.common.io.{ByteStreams, Files}
+import org.apache.hadoop.yarn.api.records.{FinalApplicationStatus, YarnApplicationState}
+import org.apache.hadoop.yarn.client.api.YarnClient
 import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.ConverterUtils
 import org.scalatest.Matchers
 import org.scalatest.concurrent.Eventually._
+
 import org.apache.spark._
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.hadoop.yarn.client.api.YarnClient
-import org.apache.hadoop.yarn.api.records.{FinalApplicationStatus, YarnApplicationState}
-import org.apache.hadoop.yarn.util.ConverterUtils
 import org.apache.spark.deploy.yarn.config._
 import org.apache.spark.internal.Logging
 import org.apache.spark.launcher._
@@ -348,7 +350,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       eventually(timeout(30 seconds), interval(100 millis)) {
         handle.getState() should be (SparkAppHandle.State.LOST)
         var status = yarnClient.getApplicationReport(applicationId).getYarnApplicationState
-        status should not be (FinalApplicationStatus.KILLED)
+        status should not be (YarnApplicationState.KILLED)
       }
 
     } finally {

From c17f15f3994ba0cba4be63519f33ce4429adf489 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 14 Mar 2017 23:51:41 -0400
Subject: [PATCH 32/50] Use val instead of var

---
 .../spark/deploy/yarn/YarnClusterSuite.scala       | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)

diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 4187bf7f312d..b0cb31d1447b 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -267,9 +267,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       handle.getAppId() should not be (null)
       handle.getAppId() should startWith ("application_")
       handle.disconnect()
-
-      var applicationId = ConverterUtils.toApplicationId(handle.getAppId)
-      var yarnClient: YarnClient = getYarnClient
+      val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
+      val yarnClient: YarnClient = getYarnClient
       eventually(timeout(30 seconds), interval(100 millis)) {
         handle.getState() should be (SparkAppHandle.State.LOST)
         var status = yarnClient.getApplicationReport(applicationId).getFinalApplicationStatus()
@@ -307,8 +306,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       handle.getAppId() should startWith ("application_")
       handle.disconnect()
 
-      var applicationId = ConverterUtils.toApplicationId(handle.getAppId)
-      var yarnClient: YarnClient = getYarnClient
+      val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
+      val yarnClient: YarnClient = getYarnClient
       eventually(timeout(30 seconds), interval(100 millis)) {
         handle.getState() should be (SparkAppHandle.State.LOST)
         var status = yarnClient.getApplicationReport(applicationId).getFinalApplicationStatus
@@ -344,9 +343,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       handle.getAppId() should not be (null)
       handle.getAppId() should startWith ("application_")
       handle.disconnect()
-
-      var applicationId = ConverterUtils.toApplicationId(handle.getAppId)
-      var yarnClient: YarnClient = getYarnClient
+      val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
+      val yarnClient: YarnClient = getYarnClient
       eventually(timeout(30 seconds), interval(100 millis)) {
         handle.getState() should be (SparkAppHandle.State.LOST)
         var status = yarnClient.getApplicationReport(applicationId).getYarnApplicationState

From 026d026b8b186ce690399f8b768826edc0a6aefd Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Mon, 20 Mar 2017 18:32:50 -0400
Subject: [PATCH 33/50] formatting and refactoring

---
 .../org/apache/spark/deploy/SparkSubmit.scala |  4 +--
 .../spark/launcher/LauncherBackend.scala      |  9 +++--
 .../spark/launcher/SparkLauncherSuite.java    |  6 +---
 .../apache/spark/launcher/SparkAppHandle.java |  2 +-
 .../apache/spark/launcher/SparkLauncher.java  | 34 ++++++++++++-------
 .../apache/spark/launcher/package-info.java   |  8 ++---
 .../org/apache/spark/deploy/yarn/Client.scala | 17 +++++-----
 .../spark/deploy/yarn/YarnClusterSuite.scala  |  6 ++--
 8 files changed, 44 insertions(+), 42 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 2901ada53d37..4f1f9289849d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -717,10 +717,10 @@ object SparkSubmit extends CommandLineUtils {
       printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.")
     }
 
-    val sparkAppMainMethodArr = mainClass.getMethods().filter(_.getName() == "sparkMain")
+    val sparkAppMainMethodArr = mainClass.getMethods().filter{_.getName() == "sparkMain"}
     val isSparkApp = sparkAppMainMethodArr.length > 0
 
-    val childSparkConf = sysProps.filter( p => p._1.startsWith("spark.")).toMap
+    val childSparkConf = sysProps.filter{p => p._1.startsWith("spark.")}.toMap
 
     // If running a SparkApp we can explicitly pass in the confs separately.
     // If we aren't running a SparkApp they get passed via the system properties.
diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
index 8c164cba3d14..f6a141e3ce07 100644
--- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
@@ -58,17 +58,16 @@ private[spark] abstract class LauncherBackend extends Logging {
     _isConnected = true
     if (stopOnShutdown) {
       logDebug("Adding shutdown hook") // force eager creation of logger
-      var _shutdownHookRef = ShutdownHookManager.addShutdownHook(
+      ShutdownHookManager.addShutdownHook(
         ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () =>
         logInfo("Invoking onStopRequest() from shutdown hook")
         try {
           if (_isConnected) {
             onStopRequest()
           }
-        }
-        catch {
-          case anotherIOE: IOException =>
-            logError("Error while running LauncherBackend shutdownHook...", anotherIOE)
+        } catch {
+          case ioException: IOException =>
+            logError("Error while running LauncherBackend shutdownHook...", ioException)
         }
       }
     }
diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
index 6b1f1c7d589e..e7a761a59b1e 100644
--- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
+++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -175,7 +175,7 @@ public void testChildProcLauncher() throws Exception {
       .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path"))
       .addSparkArg(opts.CLASS, "ShouldBeOverriddenBelow")
       .setMainClass(SparkLauncherTestApp.class.getName())
-      .autoShutdown()
+      .autoShutdown(true)
       .addAppArgs("proc");
     final Process app = launcher.launch();
 
@@ -197,15 +197,11 @@ public void testThreadLauncher() throws Exception {
     launcher
       .setMaster("local")
       .setAppResource(SparkLauncher.NO_RESOURCE)
-      .addSparkArg(opts.CONF,
-        String.format("%s=-Dfoo=ShouldBeOverriddenBelow", SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS))
       .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS,
         "-Dfoo=bar -Dtest.appender=childproc")
       .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path"))
-      .addSparkArg(opts.CLASS, "ShouldBeOverriddenBelow")
       .setMainClass(SparkLauncherTestApp.class.getName())
       .launchAsThread(true)
-      .autoShutdown()
       .addAppArgs("proc");
     final Process app = launcher.launch();
 
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
index d021649aeefb..253b4aaabb91 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
@@ -96,7 +96,7 @@ public boolean isFinal() {
 
   /**
    * Disconnects the handle from the application. If using {@link SparkLauncher#autoShutdown()}
-   * option, this method would shutdown stop/kill the application. After this method is called,
+   * option, this method will also stop the child Spark application. After this method is called,
    * the handle will not be able to communicate with the application anymore.
    */
   void disconnect();
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index d92c5216e56f..4ec054f4b498 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -101,7 +101,7 @@ public class SparkLauncher {
 
   static final Map launcherConfig = new HashMap<>();
 
-  private boolean stopOnShutdown = false;
+  private boolean autoShutdown = false;
 
   /** Flag to decide on launching spark-submit as a child process or a thread **/
   private boolean launchAsThread = false;
@@ -119,15 +119,18 @@ public static void setConfig(String name, String value) {
     launcherConfig.put(name, value);
   }
 
+
+
   /**
    * Specifies that Spark Application be stopped if current process goes away.
-   * It tries stop/kill Spark Application if {@link LauncherServer} goes away.
+   * It tries stop/kill Spark Application if launching process goes away.
    *
    * @since 2.2.0
+   * @param autoShutdown Flag for shutdown Spark Application if launcher process goes away.
    * @return This launcher.
    */
-  public SparkLauncher autoShutdown() {
-    this.stopOnShutdown = true;
+  public SparkLauncher autoShutdown(boolean autoShutdown) {
+    this.autoShutdown = autoShutdown;
     return this;
   }
 
@@ -136,6 +139,7 @@ public SparkLauncher autoShutdown() {
    * this feature is currently supported only for YARN cluster deployment mode.
    *
    * @since 2.2.0
+   * @param launchAsThread Flag for launching app as a thread.
    * @return This launcher.
    */
   public SparkLauncher launchAsThread(boolean launchAsThread) {
@@ -559,7 +563,8 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr
     return startApplicationAsChildProc(listeners);
   }
 
-  private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] listeners) throws IOException {
+  private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] listeners)
+      throws IOException {
     ChildProcAppHandle handle = LauncherServer.newAppHandle();
     for (SparkAppHandle.Listener l : listeners) {
       handle.addListener(l);
@@ -579,7 +584,8 @@ private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] lis
     pb.environment().put(LauncherProtocol.ENV_LAUNCHER_PORT,
       String.valueOf(LauncherServer.getServerInstance().getPort()));
     pb.environment().put(LauncherProtocol.ENV_LAUNCHER_SECRET, handle.getSecret());
-    pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_IF_SHUTDOWN, String.valueOf(stopOnShutdown));
+    pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_IF_SHUTDOWN,
+        String.valueOf(autoShutdown));
     try {
       handle.setChildProc(pb.start(), loggerName);
     } catch (IOException ioe) {
@@ -589,7 +595,8 @@ private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] lis
     return handle;
   }
 
-  private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... listeners) throws IOException {
+  private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... listeners)
+      throws IOException {
     ChildThreadAppHandle handle = LauncherServer.newAppThreadHandle();
     for (SparkAppHandle.Listener l : listeners) {
       handle.addListener(l);
@@ -598,22 +605,23 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste
     String appName = getAppName();
     setConf(LAUNCHER_INTERNAL_PORT, String.valueOf(LauncherServer.getServerInstance().getPort()));
     setConf(LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, handle.getSecret());
-    setConf(LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, String.valueOf(stopOnShutdown));
+    setConf(LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, String.valueOf(autoShutdown));
     try {
       // It is important that SparkSubmit class is available in the classpath.
       // Trying to see if method is available in the classpath else throws Exception.
       Method main = SparkSubmitRunner.getSparkSubmitMain();
-      Thread submitJobThread = new Thread(new SparkSubmitRunner(main, builder.buildSparkSubmitArgs()));
+      Thread submitJobThread = new Thread(new SparkSubmitRunner(main,
+          builder.buildSparkSubmitArgs()));
       submitJobThread.setName(appName);
       submitJobThread.setDaemon(true);
       handle.setChildThread(submitJobThread);
       submitJobThread.start();
     } catch (ClassNotFoundException cnfe) {
-      throw new IOException("Please make sure the spark jar containing SparkSubmit is in the classpath.",
-          cnfe);
+      throw new IOException("Please make sure the spark jar " +
+          "containing SparkSubmit is in the classpath.", cnfe);
     } catch (NoSuchMethodException nsme) {
-      throw new IOException("Please make sure the spark jar containing SparkSubmit version is correct.",
-          nsme);
+      throw new IOException("Please make sure the spark jar containing SparkSubmit " +
+          "version is correct.", nsme);
     }
     return handle;
   }
diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
index 9b10d2b08884..e60d699c5dc7 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
@@ -51,10 +51,10 @@
  * 
* *

- * Here is example of launching application in thread mode for yarn cluster mode - * with the stop if launcher shutdown option enabled. Showing use case of both - * {@link org.apache.spark.launcher.SparkLauncher#stopOnShutdown} and - * {@link org.apache.spark.launcher.SparkLauncher#launchAsThread} methods + * Here is example of launching application in thread mode for YARN cluster mode + * with the auto shutdown option enabled. Showing use case of both + * {@link org.apache.spark.launcher.SparkLauncher#autoShutdown()} and + * {@link org.apache.spark.launcher.SparkLauncher#launchAsThread(boolean)} methods *

* *
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 33db73cfaf44..9860ca2aab0b 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -67,9 +67,9 @@ private[spark] class Client(
   import YarnSparkHadoopUtil._
 
   def this(
-    clientArgs: ClientArguments,
-    spConf: SparkConf,
-    sysEnv: scala.collection.immutable.Map[String, String]) =
+      clientArgs: ClientArguments,
+      spConf: SparkConf,
+      sysEnv: scala.collection.immutable.Map[String, String]) =
     this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf, sysEnv)
 
   def this(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) =
@@ -83,12 +83,6 @@ private[spark] class Client(
 
   private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster"
 
-  private val launcherServerPort : Int =
-    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_PORT, "0").toInt
-  private val launcherServerSecret : String =
-    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, "")
-  private val launcherServerStopIfShutdown : Boolean =
-    sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, "false").toBoolean
   // AM related configurations
   private val amMemory = if (isClusterMode) {
     sparkConf.get(DRIVER_MEMORY).toInt
@@ -160,6 +154,11 @@ private[spark] class Client(
    */
   def submitApplication(): ApplicationId = {
     var appId: ApplicationId = null
+    val launcherServerPort: Int = sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_PORT, "0").toInt
+    val launcherServerSecret: String =
+      sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, "")
+    val launcherServerStopIfShutdown: Boolean =
+      sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, "false").toBoolean
     try {
       if (launcherServerSecret != null && launcherServerSecret != "" && launcherServerPort != 0) {
         launcherBackend.connect(
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index b0cb31d1447b..cf7dd4a7f1a3 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -210,7 +210,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     finalState should be (SparkAppHandle.State.FAILED)
   }
 
-  test("monitor app using launcher library for thread") {
+  test("monitor app running in thread using launcher library") {
     val env = new JHashMap[String, String]()
     env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
 
@@ -254,7 +254,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       .setPropertiesFile(propsFile)
       .setMaster("yarn")
       .setDeployMode("cluster")
-      .autoShutdown()
+      .autoShutdown(true)
       .setAppResource(SparkLauncher.NO_RESOURCE)
       .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
       .startApplication()
@@ -292,7 +292,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
       .setMaster("yarn")
       .setDeployMode("cluster")
       .launchAsThread(true)
-      .autoShutdown()
+      .autoShutdown(true)
       .setAppResource(SparkLauncher.NO_RESOURCE)
       .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
       .startApplication()

From fe5b5d64b56dd55ad4a956619bf41c3492975f89 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 21 Mar 2017 15:18:58 -0400
Subject: [PATCH 34/50] Refactor tests code

---
 .../spark/deploy/yarn/YarnClusterSuite.scala  | 154 +++++-------------
 1 file changed, 42 insertions(+), 112 deletions(-)

diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index cf7dd4a7f1a3..76976d43cae6 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -38,8 +38,9 @@ import org.apache.spark._
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.deploy.yarn.config._
 import org.apache.spark.internal.Logging
-import org.apache.spark.launcher._
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded}
+import org.apache.spark.launcher.{SparkAppHandle, _}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart,
+  SparkListenerExecutorAdded}
 import org.apache.spark.scheduler.cluster.ExecutorInfo
 import org.apache.spark.tags.ExtendedYarnTest
 import org.apache.spark.util.Utils
@@ -171,29 +172,10 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
   }
 
   test("monitor app using launcher library") {
-    val env = new JHashMap[String, String]()
-    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
-
-    val propsFile = createConfFile()
-    val handle = new SparkLauncher(env)
-      .setSparkHome(sys.props("spark.test.home"))
-      .setConf("spark.ui.enabled", "false")
-      .setPropertiesFile(propsFile)
-      .setMaster("yarn")
-      .setDeployMode("client")
-      .setAppResource(SparkLauncher.NO_RESOURCE)
-      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
-      .startApplication()
-
+    var handle : SparkAppHandle = null
     try {
-      eventually(timeout(30 seconds), interval(100 millis)) {
-        handle.getState() should be (SparkAppHandle.State.RUNNING)
-      }
-
-      handle.getAppId() should not be (null)
-      handle.getAppId() should startWith ("application_")
+      handle = launchSparkAppWithConf(false, false, "client")
       handle.stop()
-
       eventually(timeout(30 seconds), interval(100 millis)) {
         handle.getState() should be (SparkAppHandle.State.KILLED)
       }
@@ -211,28 +193,9 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
   }
 
   test("monitor app running in thread using launcher library") {
-    val env = new JHashMap[String, String]()
-    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
-
-    val propsFile = createConfFile()
-    val handle = new SparkLauncher(env)
-      .setSparkHome(sys.props("spark.test.home"))
-      .setConf("spark.ui.enabled", "false")
-      .setPropertiesFile(propsFile)
-      .setMaster("yarn")
-      .setDeployMode("cluster")
-      .launchAsThread(true)
-      .setAppResource(SparkLauncher.NO_RESOURCE)
-      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
-      .startApplication()
-
+    var handle : SparkAppHandle = null
     try {
-      eventually(timeout(30 seconds), interval(100 millis)) {
-        handle.getState() should be (SparkAppHandle.State.RUNNING)
-      }
-
-      handle.getAppId() should not be (null)
-      handle.getAppId() should startWith ("application_")
+      handle = launchSparkAppWithConf(true, false, "cluster")
       handle.stop()
 
       eventually(timeout(30 seconds), interval(100 millis)) {
@@ -244,28 +207,9 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
   }
 
   test("monitor app using launcher library for proc with auto shutdown") {
-    val env = new JHashMap[String, String]()
-    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
-
-    val propsFile = createConfFile()
-    val handle = new SparkLauncher(env)
-      .setSparkHome(sys.props("spark.test.home"))
-      .setConf("spark.ui.enabled", "false")
-      .setPropertiesFile(propsFile)
-      .setMaster("yarn")
-      .setDeployMode("cluster")
-      .autoShutdown(true)
-      .setAppResource(SparkLauncher.NO_RESOURCE)
-      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
-      .startApplication()
-
+    var handle : SparkAppHandle = null
     try {
-      eventually(timeout(30 seconds), interval(100 millis)) {
-        handle.getState() should be (SparkAppHandle.State.RUNNING)
-      }
-
-      handle.getAppId() should not be (null)
-      handle.getAppId() should startWith ("application_")
+      handle = launchSparkAppWithConf(false, true, "cluster")
       handle.disconnect()
       val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
       val yarnClient: YarnClient = getYarnClient
@@ -274,38 +218,16 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
         var status = yarnClient.getApplicationReport(applicationId).getFinalApplicationStatus()
         status should be (FinalApplicationStatus.KILLED)
       }
-
     } finally {
       handle.kill()
     }
   }
 
   test("monitor app using launcher library for thread with auto shutdown") {
-    val env = new JHashMap[String, String]()
-    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
-
-    val propsFile = createConfFile()
-    val handle = new SparkLauncher(env)
-      .setSparkHome(sys.props("spark.test.home"))
-      .setConf("spark.ui.enabled", "false")
-      .setPropertiesFile(propsFile)
-      .setMaster("yarn")
-      .setDeployMode("cluster")
-      .launchAsThread(true)
-      .autoShutdown(true)
-      .setAppResource(SparkLauncher.NO_RESOURCE)
-      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
-      .startApplication()
-
+    var handle : SparkAppHandle = null
     try {
-      eventually(timeout(30 seconds), interval(100 millis)) {
-        handle.getState() should be (SparkAppHandle.State.RUNNING)
-      }
-
-      handle.getAppId() should not be (null)
-      handle.getAppId() should startWith ("application_")
+      handle = launchSparkAppWithConf(true, true, "cluster")
       handle.disconnect()
-
       val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
       val yarnClient: YarnClient = getYarnClient
       eventually(timeout(30 seconds), interval(100 millis)) {
@@ -313,35 +235,15 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
         var status = yarnClient.getApplicationReport(applicationId).getFinalApplicationStatus
         status should be (FinalApplicationStatus.KILLED)
       }
-
     } finally {
       handle.kill()
     }
   }
 
   test("monitor app using launcher library for thread without auto shutdown") {
-    val env = new JHashMap[String, String]()
-    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
-
-    val propsFile = createConfFile()
-    val handle = new SparkLauncher(env)
-      .setSparkHome(sys.props("spark.test.home"))
-      .setConf("spark.ui.enabled", "false")
-      .setPropertiesFile(propsFile)
-      .setMaster("yarn")
-      .setDeployMode("cluster")
-      .launchAsThread(true)
-      .setAppResource(SparkLauncher.NO_RESOURCE)
-      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
-      .startApplication()
-
+    var handle : SparkAppHandle = null
     try {
-      eventually(timeout(30 seconds), interval(100 millis)) {
-        handle.getState() should be (SparkAppHandle.State.RUNNING)
-      }
-
-      handle.getAppId() should not be (null)
-      handle.getAppId() should startWith ("application_")
+      handle = launchSparkAppWithConf(true, false, "cluster")
       handle.disconnect()
       val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
       val yarnClient: YarnClient = getYarnClient
@@ -350,7 +252,6 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
         var status = yarnClient.getApplicationReport(applicationId).getYarnApplicationState
         status should not be (YarnApplicationState.KILLED)
       }
-
     } finally {
       handle.kill()
     }
@@ -445,6 +346,35 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     checkResult(finalState, executorResult, "OVERRIDDEN")
   }
 
+  private def launchSparkAppWithConf(
+      launchAsThread: Boolean,
+      autoShutdown: Boolean,
+      deployMode: String): SparkAppHandle = {
+    val env = new JHashMap[String, String]()
+    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
+
+    val propsFile = createConfFile()
+    val handle = new SparkLauncher(env)
+      .setSparkHome(sys.props("spark.test.home"))
+      .setConf("spark.ui.enabled", "false")
+      .setPropertiesFile(propsFile)
+      .setMaster("yarn")
+      .setDeployMode(deployMode)
+      .launchAsThread(launchAsThread)
+      .autoShutdown(autoShutdown)
+      .setAppResource(SparkLauncher.NO_RESOURCE)
+      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
+      .startApplication()
+
+    eventually(timeout(30 seconds), interval(100 millis)) {
+      handle.getState() should be(SparkAppHandle.State.RUNNING)
+    }
+
+    handle.getAppId() should not be (null)
+    handle.getAppId() should startWith("application_")
+    handle
+  }
+
 }
 
 private[spark] class SaveExecutorInfo extends SparkListener {

From 677edf7f091beab92e1ca42d24a20ad4c5254002 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Wed, 22 Mar 2017 14:11:04 -0400
Subject: [PATCH 35/50] Refactor launcher variables to package private

---
 .../apache/spark/launcher/SparkLauncher.java  |  6 +++---
 .../org/apache/spark/deploy/yarn/Client.scala | 16 ++------------
 .../launcher/YarnCommandBuilderUtils.scala    | 21 +++++++++++++++++--
 3 files changed, 24 insertions(+), 19 deletions(-)

diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 4ec054f4b498..29dc164a8c20 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -74,11 +74,11 @@ public class SparkLauncher {
   /** Logger name to use when launching a child process. */
   public static final String CHILD_PROCESS_LOGGER_NAME = "spark.launcher.childProcLoggerName";
 
-  public static final String LAUNCHER_INTERNAL_PORT = "spark.launcher.internal.port";
+  static final String LAUNCHER_INTERNAL_PORT = "spark.launcher.internal.port";
 
-  public static final String LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET = "spark.launcher.internal.secret";
+  static final String LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET = "spark.launcher.internal.secret";
 
-  public static final String LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN = "spark.launcher.internal.stop.on.shutdown";
+  static final String LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN = "spark.launcher.internal.stop.on.shutdown";
 
   /**
    * A special value for the resource that tells Spark to not try to process the app resource as a
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 9860ca2aab0b..98471b2bc7c3 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -52,7 +52,7 @@ import org.apache.spark.deploy.yarn.config._
 import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
-import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, SparkLauncher, YarnCommandBuilderUtils}
+import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
 import org.apache.spark.util.{CallerContext, Utils}
 
 
@@ -154,20 +154,8 @@ private[spark] class Client(
    */
   def submitApplication(): ApplicationId = {
     var appId: ApplicationId = null
-    val launcherServerPort: Int = sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_PORT, "0").toInt
-    val launcherServerSecret: String =
-      sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, "")
-    val launcherServerStopIfShutdown: Boolean =
-      sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, "false").toBoolean
     try {
-      if (launcherServerSecret != null && launcherServerSecret != "" && launcherServerPort != 0) {
-        launcherBackend.connect(
-          launcherServerPort,
-          launcherServerSecret,
-          launcherServerStopIfShutdown)
-      } else {
-        launcherBackend.connect()
-      }
+      YarnCommandBuilderUtils.launcherBackendConnect(launcherBackend, sparkConf)
       // Setup the credentials before doing anything else,
       // so we have don't have issues at any point.
       setupCredentials()
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
index 0c3d080cca25..70a562a9d3bd 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
@@ -17,10 +17,11 @@
 
 package org.apache.spark.launcher
 
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
 import scala.util.Properties
 
+import org.apache.spark.SparkConf
+
+
 /**
  * Exposes methods from the launcher library that are used by the YARN backend.
  */
@@ -38,4 +39,20 @@ private[spark] object YarnCommandBuilderUtils {
     CommandBuilderUtils.findJarsDir(sparkHome, scalaVer, true)
   }
 
+  def launcherBackendConnect(launcherBackend: LauncherBackend, sparkConf: SparkConf): Unit = {
+    val launcherServerPort: Int = sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_PORT, "0").toInt
+    val launcherServerSecret: String =
+      sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET, "")
+    val launcherServerStopIfShutdown: Boolean =
+      sparkConf.get(SparkLauncher.LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, "false").toBoolean
+    if (launcherServerSecret != null && launcherServerSecret != "" && launcherServerPort != 0) {
+      launcherBackend.connect(
+        launcherServerPort,
+        launcherServerSecret,
+        launcherServerStopIfShutdown)
+    } else {
+      launcherBackend.connect()
+    }
+  }
+
 }

From ee3f24adaa00e9323707c6ad64f3a4eb2d4f571e Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 28 Mar 2017 10:46:18 -0400
Subject: [PATCH 36/50] Fixing compilation errors

---
 .../org/apache/spark/deploy/yarn/ApplicationMaster.scala     | 3 ++-
 .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +-
 .../src/main/scala/org/apache/spark/deploy/yarn/config.scala | 5 -----
 .../org/apache/spark/launcher/YarnCommandBuilderUtils.scala  | 5 +++++
 4 files changed, 8 insertions(+), 7 deletions(-)

diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index c41673305cb7..22fe4d66743a 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -41,6 +41,7 @@ import org.apache.spark.deploy.yarn.config._
 import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, ConfigurableCredentialManager}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
+import org.apache.spark.launcher.YarnCommandBuilderUtils
 import org.apache.spark.rpc._
 import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend}
 import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
@@ -175,7 +176,7 @@ private[spark] class ApplicationMaster(
     }
 
     // Clean up the spark launcher configuration so it doesn't show up in the Web UI for security.
-    LAUNCHER_CONFIGS.foreach { e =>
+    YarnCommandBuilderUtils.LAUNCHER_CONFIGS.foreach { e =>
       sparkConf.remove(e)
       sys.props.remove(e)
     }
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 98471b2bc7c3..abbc60a9532c 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -646,7 +646,7 @@ private[spark] class Client(
     // polluting the UI's environment page. This works for client mode; for cluster mode, this
     // is handled by the AM.
     CACHE_CONFIGS.foreach(sparkConf.remove)
-    LAUNCHER_CONFIGS.foreach(sparkConf.remove)
+    YarnCommandBuilderUtils.LAUNCHER_CONFIGS.foreach(sparkConf.remove)
 
     localResources
   }
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
index 5856f5a3d860..0349eb20d558 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -356,9 +356,4 @@ package object config {
     CACHED_FILES_VISIBILITIES,
     CACHED_FILES_TYPES,
     CACHED_CONF_ARCHIVE)
-
-  private[yarn] val LAUNCHER_CONFIGS = Seq(
-    SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET,
-    SparkLauncher.LAUNCHER_INTERNAL_PORT)
-
 }
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
index 70a562a9d3bd..f257201b6134 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
@@ -55,4 +55,9 @@ private[spark] object YarnCommandBuilderUtils {
     }
   }
 
+  private[yarn] val LAUNCHER_CONFIGS = Seq(
+    SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET,
+    SparkLauncher.LAUNCHER_INTERNAL_PORT)
+
+
 }

From 30b460cc488bc5519ee698627e941eed98928189 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 28 Mar 2017 11:01:22 -0400
Subject: [PATCH 37/50] Fixing compilation errors

---
 .../org/apache/spark/launcher/YarnCommandBuilderUtils.scala   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
index f257201b6134..2f99de50d170 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
@@ -55,9 +55,9 @@ private[spark] object YarnCommandBuilderUtils {
     }
   }
 
-  private[yarn] val LAUNCHER_CONFIGS = Seq(
+  private[spark] val LAUNCHER_CONFIGS = Seq(
     SparkLauncher.LAUNCHER_INTERNAL_CHILD_PROCESS_SECRET,
     SparkLauncher.LAUNCHER_INTERNAL_PORT)
 
-
 }
+

From 7323200e4e70ab9b8fbbe8b69b1fbe8ac9c24512 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Tue, 4 Apr 2017 14:42:24 -0400
Subject: [PATCH 38/50] Addressinng code review comments

---
 .../src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +-
 .../java/org/apache/spark/launcher/SparkLauncherSuite.java   | 5 -----
 .../main/java/org/apache/spark/launcher/SparkLauncher.java   | 4 ++--
 .../main/java/org/apache/spark/launcher/package-info.java    | 4 ++--
 .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 1 -
 .../src/main/scala/org/apache/spark/deploy/yarn/config.scala | 2 +-
 6 files changed, 6 insertions(+), 12 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 4f1f9289849d..96a3e2ad81bb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -717,7 +717,7 @@ object SparkSubmit extends CommandLineUtils {
       printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.")
     }
 
-    val sparkAppMainMethodArr = mainClass.getMethods().filter{_.getName() == "sparkMain"}
+    val sparkAppMainMethodArr = mainClass.getMethods().filter(_.getName == "sparkMain")
     val isSparkApp = sparkAppMainMethodArr.length > 0
 
     val childSparkConf = sysProps.filter{p => p._1.startsWith("spark.")}.toMap
diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
index e7a761a59b1e..613529d38c80 100644
--- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
+++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -175,7 +175,6 @@ public void testChildProcLauncher() throws Exception {
       .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path"))
       .addSparkArg(opts.CLASS, "ShouldBeOverriddenBelow")
       .setMainClass(SparkLauncherTestApp.class.getName())
-      .autoShutdown(true)
       .addAppArgs("proc");
     final Process app = launcher.launch();
 
@@ -190,10 +189,6 @@ public void testThreadLauncher() throws Exception {
     // by the path length limitation. See SPARK-18718.
     assumeTrue(!Utils.isWindows());
 
-    SparkSubmitOptionParser opts = new SparkSubmitOptionParser();
-    Map env = new HashMap<>();
-    env.put("SPARK_PRINT_LAUNCH_COMMAND", "1");
-
     launcher
       .setMaster("local")
       .setAppResource(SparkLauncher.NO_RESOURCE)
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 29dc164a8c20..e5b9154e6eec 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -620,8 +620,8 @@ private SparkAppHandle startApplicationAsThread(SparkAppHandle.Listener... liste
       throw new IOException("Please make sure the spark jar " +
           "containing SparkSubmit is in the classpath.", cnfe);
     } catch (NoSuchMethodException nsme) {
-      throw new IOException("Please make sure the spark jar containing SparkSubmit " +
-          "version is correct.", nsme);
+      throw new IOException("Please make sure the version of the Spark jar containing " +
+          "SparkSubmit is correct.", nsme);
     }
     return handle;
   }
diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
index e60d699c5dc7..527d735b3ded 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
@@ -53,14 +53,14 @@
  * 

* Here is example of launching application in thread mode for YARN cluster mode * with the auto shutdown option enabled. Showing use case of both - * {@link org.apache.spark.launcher.SparkLauncher#autoShutdown()} and + * {@link org.apache.spark.launcher.SparkLauncher#autoShutdown(boolean)} and * {@link org.apache.spark.launcher.SparkLauncher#launchAsThread(boolean)} methods *

* *
  * {@code
  *   import org.apache.spark.launcher.SparkAppHandle;
- *   import org.apache.spark.launcher.SparkLauncher;`
+ *   import org.apache.spark.launcher.SparkLauncher;
  *
  *   public class MyLauncher {
  *     public static void main(String[] args) throws Exception {
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index abbc60a9532c..e390f930cf27 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -55,7 +55,6 @@ import org.apache.spark.internal.config._
 import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
 import org.apache.spark.util.{CallerContext, Utils}
 
-
 private[spark] class Client(
     val args: ClientArguments,
     val hadoopConf: Configuration,
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
index 0349eb20d558..d8c96c35ca71 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -20,7 +20,6 @@ package org.apache.spark.deploy.yarn
 import java.util.concurrent.TimeUnit
 
 import org.apache.spark.internal.config.ConfigBuilder
-import org.apache.spark.launcher.SparkLauncher
 import org.apache.spark.network.util.ByteUnit
 
 package object config {
@@ -356,4 +355,5 @@ package object config {
     CACHED_FILES_VISIBILITIES,
     CACHED_FILES_TYPES,
     CACHED_CONF_ARCHIVE)
+
 }

From 0cfd4a7eb540e751a03b9d8e78af4e8f6e3be62c Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Thu, 6 Apr 2017 17:08:00 -0400
Subject: [PATCH 39/50] Remove sys.env reference and passing it to SparkApp

---
 core/src/main/scala/org/apache/spark/SparkApp.scala          | 4 +---
 .../src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 3 +--
 .../main/java/org/apache/spark/launcher/SparkLauncher.java   | 1 +
 .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 5 ++---
 4 files changed, 5 insertions(+), 8 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala
index 97298197c271..f7ec0be64a35 100644
--- a/core/src/main/scala/org/apache/spark/SparkApp.scala
+++ b/core/src/main/scala/org/apache/spark/SparkApp.scala
@@ -30,11 +30,9 @@ private[spark] trait SparkApp {
    *
    * @param args    - all arguments for SparkApp.
    * @param conf    - Spark Configuration.
-   * @param envvars - system environment Variables.
    */
   def sparkMain(
     args: Array[String],
-    conf: Map[String, String],
-    envvars: Map[String, String]): Unit
+    conf: Map[String, String]): Unit
 
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 96a3e2ad81bb..d5003084a516 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -751,8 +751,7 @@ object SparkSubmit extends CommandLineUtils {
 
     try {
       if (isSparkApp) {
-        val envvars = Map[String, String]() ++ sys.env
-        mainMethod.invoke(null, childArgs.toArray, childSparkConf, envvars.toMap)
+        mainMethod.invoke(null, childArgs.toArray, childSparkConf)
       } else {
         mainMethod.invoke(null, childArgs.toArray)
       }
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index e5b9154e6eec..c82202c69a06 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -586,6 +586,7 @@ private SparkAppHandle startApplicationAsChildProc(SparkAppHandle.Listener[] lis
     pb.environment().put(LauncherProtocol.ENV_LAUNCHER_SECRET, handle.getSecret());
     pb.environment().put(LauncherProtocol.ENV_LAUNCHER_STOP_IF_SHUTDOWN,
         String.valueOf(autoShutdown));
+    setConf(LAUNCHER_INTERNAL_STOP_ON_SHUTDOWN, String.valueOf(autoShutdown));
     try {
       handle.setChildProc(pb.start(), loggerName);
     } catch (IOException ioe) {
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index e390f930cf27..24e9a1ba48d0 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -1142,8 +1142,7 @@ private object Client extends SparkApp with Logging {
 
   override def sparkMain(
     args: Array[String],
-    conf: scala.collection.immutable.Map[String, String],
-    envvars: scala.collection.immutable.Map[String, String]): Unit = {
+    conf: scala.collection.immutable.Map[String, String]): Unit = {
     if (!sys.props.contains("SPARK_SUBMIT")) {
       logWarning("WARNING: This client is deprecated and will be removed in a " +
         "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"")
@@ -1163,7 +1162,7 @@ private object Client extends SparkApp with Logging {
     sparkConf.remove("spark.jars")
     sparkConf.remove("spark.files")
     val argsForClient = new ClientArguments(args)
-    new Client(argsForClient, sparkConf, envvars).run()
+    new Client(argsForClient, sparkConf).run()
   }
 
   // Alias for the user jar

From 8609874ee0c163b509a49fcb70f9f827dcbb1dbe Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Wed, 19 Apr 2017 17:06:36 -0400
Subject: [PATCH 40/50] Removing changes related to passing sys.env

---
 .../scala/org/apache/spark/SparkApp.scala     | 11 ++---
 .../org/apache/spark/deploy/SparkSubmit.scala | 27 +++++-------
 .../apache/spark/launcher/SparkLauncher.java  |  2 +
 .../spark/launcher/SparkSubmitRunner.java     |  6 ++-
 .../apache/spark/launcher/package-info.java   |  2 +-
 .../org/apache/spark/deploy/yarn/Client.scala | 42 +++++--------------
 6 files changed, 32 insertions(+), 58 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkApp.scala b/core/src/main/scala/org/apache/spark/SparkApp.scala
index f7ec0be64a35..9d8bb01070fe 100644
--- a/core/src/main/scala/org/apache/spark/SparkApp.scala
+++ b/core/src/main/scala/org/apache/spark/SparkApp.scala
@@ -18,21 +18,18 @@
 package org.apache.spark
 
 /**
- * A client that SparkSubmit uses to launch spark Application.
- * This is currently supported only in YARN mode.
+ * An interface that can be implemented by applications launched by SparkSubmit
+ * which exposes the Spark job configuration explicitly.
  */
 private[spark] trait SparkApp {
   this: Singleton =>
 
   /**
-   * The Client should implement this as entry method to provide application,
-   * spark conf and system configuration.
+   * Method executed by SparkSubmit to run the application.
    *
    * @param args    - all arguments for SparkApp.
    * @param conf    - Spark Configuration.
    */
-  def sparkMain(
-    args: Array[String],
-    conf: Map[String, String]): Unit
+  def sparkMain(args: Array[String], conf: Map[String, String]): Unit
 
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index d5003084a516..37a0dcd84e91 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -717,28 +717,17 @@ object SparkSubmit extends CommandLineUtils {
       printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.")
     }
 
-    val sparkAppMainMethodArr = mainClass.getMethods().filter(_.getName == "sparkMain")
-    val isSparkApp = sparkAppMainMethodArr.length > 0
-
-    val childSparkConf = sysProps.filter{p => p._1.startsWith("spark.")}.toMap
+    val sparkAppMainMethod = mainClass.getMethods().find(_.getName == "sparkMain")
+    val childSparkConf = sysProps.filter{ p => p._1.startsWith("spark.") }.toMap
 
     // If running a SparkApp we can explicitly pass in the confs separately.
     // If we aren't running a SparkApp they get passed via the system properties.
-    if (!isSparkApp) {
+    if (sparkAppMainMethod.isEmpty) {
       sysProps.foreach { case (key, value) =>
         System.setProperty(key, value)
       }
     }
 
-    val mainMethod = if (isSparkApp) {
-      sparkAppMainMethodArr(0)
-    } else {
-      mainClass.getMethod("main", new Array[String](0).getClass)
-    }
-    if (!Modifier.isStatic(mainMethod.getModifiers)) {
-      throw new IllegalStateException("The main method in the given main class must be static")
-    }
-
     @tailrec
     def findCause(t: Throwable): Throwable = t match {
       case e: UndeclaredThrowableException =>
@@ -750,9 +739,15 @@ object SparkSubmit extends CommandLineUtils {
     }
 
     try {
-      if (isSparkApp) {
-        mainMethod.invoke(null, childArgs.toArray, childSparkConf)
+      if (sparkAppMainMethod.isDefined) {
+        sparkAppMainMethod.get.invoke(null, childArgs.toArray, childSparkConf)
       } else {
+        val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass)
+
+        if (!Modifier.isStatic(mainMethod.getModifiers)) {
+          throw new IllegalStateException("The main method in the given main class must be static")
+        }
+
         mainMethod.invoke(null, childArgs.toArray)
       }
     } catch {
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index c82202c69a06..72c122153372 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -558,6 +558,8 @@ private String getAppName() throws IOException {
    */
   public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) throws IOException {
     if (launchAsThread) {
+      checkArgument(builder.childEnv.isEmpty(),
+        "Environment variables are not supported while launching as Thread");
       return startApplicationAsThread(listeners);
     }
     return startApplicationAsChildProc(listeners);
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
index f2af68cf3d8d..d7ec8a4001a0 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitRunner.java
@@ -57,8 +57,10 @@ public void run() {
       }
       Object argsObj = args.toArray(new String[args.size()]);
       main.invoke(null, argsObj);
-    } catch (IllegalAccessException|InvocationTargetException|ClassNotFoundException|NoSuchMethodException illEx) {
-      throw new RuntimeException(illEx);
+    } catch (RuntimeException runtimeException) {
+      throw runtimeException;
+    }  catch (Exception exception){
+      throw new RuntimeException(exception);
     }
   }
 }
diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
index 527d735b3ded..9b22b123d1fc 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
@@ -69,7 +69,7 @@
  *         .setMainClass("my.spark.app.Main")
  *         .setMaster("yarn")
  *         .setDeployMode("cluster")
- *         .autoShutdown()
+ *         .autoShutdown(true)
  *         .launchAsThread(true)
  *         .setConf(SparkLauncher.DRIVER_MEMORY, "2g")
  *         .startApplication();
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 24e9a1ba48d0..4859ffca2e71 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -58,24 +58,14 @@ import org.apache.spark.util.{CallerContext, Utils}
 private[spark] class Client(
     val args: ClientArguments,
     val hadoopConf: Configuration,
-    val sparkConf: SparkConf,
-    val sysEnvironment: scala.collection.immutable.Map[String, String])
+    val sparkConf: SparkConf)
   extends Logging {
 
   import Client._
   import YarnSparkHadoopUtil._
 
-  def this(
-      clientArgs: ClientArguments,
-      spConf: SparkConf,
-      sysEnv: scala.collection.immutable.Map[String, String]) =
-    this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf, sysEnv)
-
-  def this(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) =
-    this(clientArgs, hadoopConf, spConf, sys.env.toMap)
-
   def this(clientArgs: ClientArguments, spConf: SparkConf) =
-    this(clientArgs, spConf, sys.env.toMap)
+    this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf)
 
   private val yarnClient = YarnClient.createYarnClient
   private val yarnConf = new YarnConfiguration(hadoopConf)
@@ -686,7 +676,7 @@ private[spark] class Client(
     }
 
     Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
-      sysEnvironment.get(envKey).foreach { path =>
+      sys.env.get(envKey).foreach { path =>
         val dir = new File(path)
         if (dir.isDirectory()) {
           val files = dir.listFiles()
@@ -742,8 +732,7 @@ private[spark] class Client(
       pySparkArchives: Seq[String]): HashMap[String, String] = {
     logInfo("Setting up the launch environment for our AM container")
     val env = new HashMap[String, String]()
-    populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH),
-      sysEnvironment)
+    populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH))
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString
     env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName()
@@ -790,13 +779,13 @@ private[spark] class Client(
       // propagate PYSPARK_DRIVER_PYTHON and PYSPARK_PYTHON to driver in cluster mode
       Seq("PYSPARK_DRIVER_PYTHON", "PYSPARK_PYTHON").foreach { envname =>
         if (!env.contains(envname)) {
-          sysEnvironment.get(envname).foreach(env(envname) = _)
+          sys.env.get(envname).foreach(env(envname) = _)
         }
       }
       sys.env.get("PYTHONHASHSEED").foreach(env.put("PYTHONHASHSEED", _))
     }
 
-    sysEnvironment.get(ENV_DIST_CLASSPATH).foreach { dcp =>
+    sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp =>
       env(ENV_DIST_CLASSPATH) = dcp
     }
 
@@ -1122,10 +1111,10 @@ private[spark] class Client(
   }
 
   private def findPySparkArchives(): Seq[String] = {
-    sysEnvironment.get("PYSPARK_ARCHIVES_PATH")
+    sys.env.get("PYSPARK_ARCHIVES_PATH")
       .map(_.split(",").toSeq)
       .getOrElse {
-        val pyLibPath = Seq(sysEnvironment("SPARK_HOME"), "python", "lib").mkString(File.separator)
+        val pyLibPath = Seq(sys.env("SPARK_HOME"), "python", "lib").mkString(File.separator)
         val pyArchivesFile = new File(pyLibPath, "pyspark.zip")
         require(pyArchivesFile.exists(),
           s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.")
@@ -1238,15 +1227,6 @@ private object Client extends SparkApp with Logging {
   private[yarn] def getDefaultMRApplicationClasspath: Seq[String] =
     StringUtils.getStrings(MRJobConfig.DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH).toSeq
 
-  private[yarn] def populateClasspath(
-      args: ClientArguments,
-      conf: Configuration,
-      sparkConf: SparkConf,
-      env: HashMap[String, String],
-      extraClassPath: Option[String]): Unit = {
-    populateClasspath(args, conf, sparkConf, env, extraClassPath, sys.env)
-  }
-
   /**
    * Populate the classpath entry in the given environment map.
    *
@@ -1262,9 +1242,7 @@ private object Client extends SparkApp with Logging {
       conf: Configuration,
       sparkConf: SparkConf,
       env: HashMap[String, String],
-      extraClassPath: Option[String] = None,
-      providedEnv: scala.collection.immutable.Map[String, String]
-        = sys.env): Unit = {
+      extraClassPath: Option[String] = None): Unit = {
     extraClassPath.foreach { cp =>
       addClasspathEntry(getClusterPath(sparkConf, cp), env)
     }
@@ -1307,7 +1285,7 @@ private object Client extends SparkApp with Logging {
     }
 
     populateHadoopClasspath(conf, env)
-    providedEnv.get(ENV_DIST_CLASSPATH).foreach { cp =>
+    sys.env.get(ENV_DIST_CLASSPATH).foreach { cp =>
       addClasspathEntry(getClusterPath(sparkConf, cp), env)
     }
   }

From 14c6365dc37ca5a3465a76040d8c099081cbf1b0 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Wed, 26 Apr 2017 09:48:25 -0400
Subject: [PATCH 41/50] Fixing documentation and review comments

---
 .../main/scala/org/apache/spark/deploy/SparkSubmit.scala   | 2 +-
 .../java/org/apache/spark/launcher/SparkAppHandle.java     | 5 +++--
 .../main/java/org/apache/spark/launcher/SparkLauncher.java | 7 ++++---
 .../apache/spark/launcher/YarnCommandBuilderUtils.scala    | 1 -
 .../org/apache/spark/deploy/yarn/YarnClusterSuite.scala    | 6 ++----
 5 files changed, 10 insertions(+), 11 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 91cadc9a416f..138f8edad084 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -723,7 +723,7 @@ object SparkSubmit extends CommandLineUtils {
     }
 
     val sparkAppMainMethod = mainClass.getMethods().find(_.getName == "sparkMain")
-    val childSparkConf = sysProps.filter{ p => p._1.startsWith("spark.") }.toMap
+    val childSparkConf = sysProps.filter { p => p._1.startsWith("spark.") }.toMap
 
     // If running a SparkApp we can explicitly pass in the confs separately.
     // If we aren't running a SparkApp they get passed via the system properties.
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
index 253b4aaabb91..1ba12078d211 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
@@ -95,8 +95,9 @@ public boolean isFinal() {
   void kill();
 
   /**
-   * Disconnects the handle from the application. If using {@link SparkLauncher#autoShutdown()}
-   * option, this method will also stop the child Spark application. After this method is called,
+   * Disconnects the handle from the application.
+   * If using {@link SparkLauncher#autoShutdown(boolean)}} option, this method will
+   * also stop the child Spark application. After this method is called,
    * the handle will not be able to communicate with the application anymore.
    */
   void disconnect();
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 72c122153372..16761e93042a 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -126,7 +126,7 @@ public static void setConfig(String name, String value) {
    * It tries stop/kill Spark Application if launching process goes away.
    *
    * @since 2.2.0
-   * @param autoShutdown Flag for shutdown Spark Application if launcher process goes away.
+   * @param autoShutdown Whether to shut down the Spark application if the launcher process goes away.
    * @return This launcher.
    */
   public SparkLauncher autoShutdown(boolean autoShutdown) {
@@ -139,7 +139,8 @@ public SparkLauncher autoShutdown(boolean autoShutdown) {
    * this feature is currently supported only for YARN cluster deployment mode.
    *
    * @since 2.2.0
-   * @param launchAsThread Flag for launching app as a thread.
+   * @param launchAsThread Whether to launch the Spark application in a new thread in
+   *                       the same process.
    * @return This launcher.
    */
   public SparkLauncher launchAsThread(boolean launchAsThread) {
@@ -559,7 +560,7 @@ private String getAppName() throws IOException {
   public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) throws IOException {
     if (launchAsThread) {
       checkArgument(builder.childEnv.isEmpty(),
-        "Environment variables are not supported while launching as Thread");
+        "Custom environment variables are not supported while launching in a Thread");
       return startApplicationAsThread(listeners);
     }
     return startApplicationAsChildProc(listeners);
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
index 2f99de50d170..fc929c6ec458 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
@@ -21,7 +21,6 @@ import scala.util.Properties
 
 import org.apache.spark.SparkConf
 
-
 /**
  * Exposes methods from the launcher library that are used by the YARN backend.
  */
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 76976d43cae6..5b17f1084b79 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -224,9 +224,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
   }
 
   test("monitor app using launcher library for thread with auto shutdown") {
-    var handle : SparkAppHandle = null
+    var handle = launchSparkAppWithConf(true, true, "cluster")
     try {
-      handle = launchSparkAppWithConf(true, true, "cluster")
       handle.disconnect()
       val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
       val yarnClient: YarnClient = getYarnClient
@@ -241,9 +240,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
   }
 
   test("monitor app using launcher library for thread without auto shutdown") {
-    var handle : SparkAppHandle = null
+    var handle = launchSparkAppWithConf(true, false, "cluster")
     try {
-      handle = launchSparkAppWithConf(true, false, "cluster")
       handle.disconnect()
       val applicationId = ConverterUtils.toApplicationId(handle.getAppId)
       val yarnClient: YarnClient = getYarnClient

From 04e56fcd2cc62b4e651b1016300a9ca5704248c2 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Thu, 27 Apr 2017 11:24:42 -0400
Subject: [PATCH 42/50] Adding check to launch as thread

---
 .../java/org/apache/spark/launcher/SparkLauncherSuite.java | 7 ++-----
 .../main/java/org/apache/spark/launcher/SparkLauncher.java | 4 +++-
 2 files changed, 5 insertions(+), 6 deletions(-)

diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
index 613529d38c80..f16d65972e30 100644
--- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
+++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -198,11 +198,8 @@ public void testThreadLauncher() throws Exception {
       .setMainClass(SparkLauncherTestApp.class.getName())
       .launchAsThread(true)
       .addAppArgs("proc");
-    final Process app = launcher.launch();
-
-    new OutputRedirector(app.getInputStream(), TF);
-    new OutputRedirector(app.getErrorStream(), TF);
-    assertEquals(0, app.waitFor());
+    final SparkAppHandle app = launcher.startApplication();
+    assertEquals(false, app.getState().isFinal());
   }
 
   public static class SparkLauncherTestApp {
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
index 16761e93042a..191378566046 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -500,6 +500,8 @@ public SparkLauncher redirectToLog(String loggerName) {
    * @return A process handle for the Spark app.
    */
   public Process launch() throws IOException {
+    checkArgument(launchAsThread == false,
+        "Use StartApplication method to launch application in a Thread");
     Process childProc = createBuilder().start();
     if (redirectToLog) {
       String loggerName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME);
@@ -509,7 +511,7 @@ public Process launch() throws IOException {
   }
 
   private String getAppName() throws IOException {
-    String appName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME);
+    String appName = launchAsThread ? null : builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME);
     if (appName == null) {
       if (builder.appName != null) {
         appName = builder.appName;

From 7ee465fa45bb3537bc819fb3249d3da0d2826803 Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Fri, 28 Apr 2017 11:48:42 -0400
Subject: [PATCH 43/50] Fixing breaking tests

---
 .../java/org/apache/spark/launcher/SparkLauncherSuite.java   | 1 +
 .../org/apache/spark/deploy/yarn/YarnClusterSuite.scala      | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
index f16d65972e30..9626c0207a1c 100644
--- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
+++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -189,6 +189,7 @@ public void testThreadLauncher() throws Exception {
     // by the path length limitation. See SPARK-18718.
     assumeTrue(!Utils.isWindows());
 
+    launcher = new SparkLauncher();
     launcher
       .setMaster("local")
       .setAppResource(SparkLauncher.NO_RESOURCE)
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 5b17f1084b79..46f2d11754bd 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -352,9 +352,10 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
 
     val propsFile = createConfFile()
-    val handle = new SparkLauncher(env)
+    val launcher = if (launchAsThread == true) new SparkLauncher() else new SparkLauncher(env)
       .setSparkHome(sys.props("spark.test.home"))
-      .setConf("spark.ui.enabled", "false")
+
+    val handle = launcher.setConf("spark.ui.enabled", "false")
       .setPropertiesFile(propsFile)
       .setMaster("yarn")
       .setDeployMode(deployMode)

From 3f060b66f32f5d8b3dc3c8cb8de63a4afd51384d Mon Sep 17 00:00:00 2001
From: Kishor Patil 
Date: Fri, 28 Apr 2017 13:38:30 -0400
Subject: [PATCH 44/50] Fix documentation

---
 .../src/main/java/org/apache/spark/launcher/package-info.java   | 2 --
 1 file changed, 2 deletions(-)

diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
index 9b22b123d1fc..05abba136037 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
@@ -21,8 +21,6 @@
  * 

* This library allows applications to launch Spark programmatically. There's only one entry * point to the library - the {@link org.apache.spark.launcher.SparkLauncher} class. - * Under YARN manager cluster mode, it supports launching in Application in thread or - * child JVM process. *

* *

From f1b49d8b7946edc484c6f20e6ed4ee26b7721e11 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 4 May 2017 10:07:20 -0400 Subject: [PATCH 45/50] Addressing review comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 1 - .../org/apache/spark/launcher/SparkLauncher.java | 10 +++++----- .../spark/deploy/yarn/BaseYarnClusterSuite.scala | 1 - .../spark/deploy/yarn/YarnClusterSuite.scala | 14 +++++--------- 4 files changed, 10 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 138f8edad084..fd4fdff5cb3e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -24,7 +24,6 @@ import java.security.PrivilegedExceptionAction import java.text.ParseException import scala.annotation.tailrec -import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import scala.util.Properties diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 191378566046..5fcf0b11874b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -135,8 +135,7 @@ public SparkLauncher autoShutdown(boolean autoShutdown) { } /** - * Specifies that Spark Submit be launched as a daemon thread. Please note - * this feature is currently supported only for YARN cluster deployment mode. + * Specifies that Spark Submit be launched as a daemon thread. * * @since 2.2.0 * @param launchAsThread Whether to launch the Spark application in a new thread in @@ -500,8 +499,8 @@ public SparkLauncher redirectToLog(String loggerName) { * @return A process handle for the Spark app. */ public Process launch() throws IOException { - checkArgument(launchAsThread == false, - "Use StartApplication method to launch application in a Thread"); + checkArgument(!launchAsThread, + "Use startApplication method to launch application in a thread"); Process childProc = createBuilder().start(); if (redirectToLog) { String loggerName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); @@ -511,7 +510,8 @@ public Process launch() throws IOException { } private String getAppName() throws IOException { - String appName = launchAsThread ? null : builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); + String appName = launchAsThread ? null + : builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); if (appName == null) { if (builder.appName != null) { appName = builder.appName; diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index c43aa92fd12a..3b6cf19a3b01 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -66,7 +66,6 @@ abstract class BaseYarnClusterSuite def newYarnConfig(): YarnConfiguration - override def beforeAll() { super.beforeAll() oldSystemProperties = SerializationUtils.clone(System.getProperties) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 46f2d11754bd..e01c039f7472 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -172,9 +172,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite { } test("monitor app using launcher library") { - var handle : SparkAppHandle = null + val handle = launchSparkAppWithConf(false, false, "client") try { - handle = launchSparkAppWithConf(false, false, "client") handle.stop() eventually(timeout(30 seconds), interval(100 millis)) { handle.getState() should be (SparkAppHandle.State.KILLED) @@ -193,11 +192,9 @@ class YarnClusterSuite extends BaseYarnClusterSuite { } test("monitor app running in thread using launcher library") { - var handle : SparkAppHandle = null + val handle = launchSparkAppWithConf(true, false, "cluster") try { - handle = launchSparkAppWithConf(true, false, "cluster") handle.stop() - eventually(timeout(30 seconds), interval(100 millis)) { handle.getState() should be (SparkAppHandle.State.KILLED) } @@ -207,9 +204,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite { } test("monitor app using launcher library for proc with auto shutdown") { - var handle : SparkAppHandle = null + val handle = launchSparkAppWithConf(false, true, "cluster") try { - handle = launchSparkAppWithConf(false, true, "cluster") handle.disconnect() val applicationId = ConverterUtils.toApplicationId(handle.getAppId) val yarnClient: YarnClient = getYarnClient @@ -224,7 +220,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { } test("monitor app using launcher library for thread with auto shutdown") { - var handle = launchSparkAppWithConf(true, true, "cluster") + val handle = launchSparkAppWithConf(true, true, "cluster") try { handle.disconnect() val applicationId = ConverterUtils.toApplicationId(handle.getAppId) @@ -240,7 +236,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { } test("monitor app using launcher library for thread without auto shutdown") { - var handle = launchSparkAppWithConf(true, false, "cluster") + val handle = launchSparkAppWithConf(true, false, "cluster") try { handle.disconnect() val applicationId = ConverterUtils.toApplicationId(handle.getAppId) From 2996fb1960914bd63ec7eda0a99b53f26a3d1c22 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Tue, 9 May 2017 14:55:03 -0400 Subject: [PATCH 46/50] Fix minor review comments --- .../main/java/org/apache/spark/launcher/SparkLauncher.java | 6 +++--- .../org/apache/spark/launcher/YarnCommandBuilderUtils.scala | 1 - 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 5fcf0b11874b..071af53c6fb4 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -125,7 +125,7 @@ public static void setConfig(String name, String value) { * Specifies that Spark Application be stopped if current process goes away. * It tries stop/kill Spark Application if launching process goes away. * - * @since 2.2.0 + * @since 2.3.0 * @param autoShutdown Whether to shut down the Spark application if the launcher process goes away. * @return This launcher. */ @@ -137,7 +137,7 @@ public SparkLauncher autoShutdown(boolean autoShutdown) { /** * Specifies that Spark Submit be launched as a daemon thread. * - * @since 2.2.0 + * @since 2.3.0 * @param launchAsThread Whether to launch the Spark application in a new thread in * the same process. * @return This launcher. @@ -562,7 +562,7 @@ private String getAppName() throws IOException { public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) throws IOException { if (launchAsThread) { checkArgument(builder.childEnv.isEmpty(), - "Custom environment variables are not supported while launching in a Thread"); + "Custom environment variables are not supported while launching in a thread"); return startApplicationAsThread(listeners); } return startApplicationAsChildProc(listeners); diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala index fc929c6ec458..89757ae1fa04 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.launcher import scala.util.Properties import org.apache.spark.SparkConf - /** * Exposes methods from the launcher library that are used by the YARN backend. */ From a311721bd2b872ddb6e72e0e688f44915d82fc7c Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 10 May 2017 11:39:21 -0400 Subject: [PATCH 47/50] Fixing SparkLauncherSuite test --- .../apache/spark/launcher/SparkLauncherSuite.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index 9626c0207a1c..1f9b93b92f9f 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -26,6 +26,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.bridge.SLF4JBridgeHandler; + +import static java.lang.Thread.sleep; import static org.junit.Assert.*; import static org.junit.Assume.*; @@ -198,8 +200,9 @@ public void testThreadLauncher() throws Exception { .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) .setMainClass(SparkLauncherTestApp.class.getName()) .launchAsThread(true) - .addAppArgs("proc"); + .addAppArgs("thread"); final SparkAppHandle app = launcher.startApplication(); + sleep(3000); assertEquals(false, app.getState().isFinal()); } @@ -207,8 +210,12 @@ public static class SparkLauncherTestApp { public static void main(String[] args) throws Exception { assertEquals(1, args.length); - assertEquals("proc", args[0]); - assertEquals("bar", System.getProperty("foo")); + if("proc".equalsIgnoreCase(args[0])) { + assertEquals("proc", args[0]); + assertEquals("bar", System.getProperty("foo")); + } else { + assertEquals(null, System.getProperty("foo")); + } assertEquals("local", System.getProperty(SparkLauncher.SPARK_MASTER)); } From d9060724b4f9a2dd3f4c9b0f77c2a09cb14b4671 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Tue, 23 May 2017 15:14:14 -0400 Subject: [PATCH 48/50] Fix SparkLauncherSuite with waitFor method. --- .../java/org/apache/spark/launcher/SparkLauncherSuite.java | 3 ++- .../org/apache/spark/launcher/AbstractSparkAppHandle.java | 2 ++ .../java/org/apache/spark/launcher/ChildProcAppHandle.java | 6 ++++-- .../org/apache/spark/launcher/ChildThreadAppHandle.java | 6 ++++++ 4 files changed, 14 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index 1f9b93b92f9f..e097eff0c97d 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -203,7 +203,8 @@ public void testThreadLauncher() throws Exception { .addAppArgs("thread"); final SparkAppHandle app = launcher.startApplication(); sleep(3000); - assertEquals(false, app.getState().isFinal()); + AbstractSparkAppHandle handle = (AbstractSparkAppHandle)app; + handle.waitFor(); } public static class SparkLauncherTestApp { diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java index 36d2557de841..182c723cbff8 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java @@ -129,4 +129,6 @@ private synchronized void fireEvent(boolean isInfoChanged) { } } } + + protected abstract void waitFor() throws InterruptedException; } diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java index fe02c77ac206..1e350f23a400 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java @@ -30,7 +30,7 @@ class ChildProcAppHandle extends AbstractSparkAppHandle { private static final Logger LOG = Logger.getLogger(ChildProcAppHandle.class.getName()); - private Process childProc; + protected Process childProc; ChildProcAppHandle(String secret, LauncherServer server) { super(server, secret); @@ -58,5 +58,7 @@ void setChildProc(Process childProc, String loggerName) { SparkLauncher.REDIRECTOR_FACTORY); } - + protected void waitFor() throws InterruptedException { + this.childProc.waitFor(); + } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java index 23185d10cca6..c44ed96e28fa 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java @@ -51,4 +51,10 @@ public synchronized void kill() { void setChildThread(Thread childThread) { this.childThread = childThread; } + + protected void waitFor() throws InterruptedException { + if(this.childThread.isAlive()) { + this.childThread.wait(); + } + } } From 81fd2974c3b6d7c11112021ae7ca5b8e4c6ee069 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 14 Jun 2017 11:29:13 -0400 Subject: [PATCH 49/50] Fixing SparkLauncherSuite unit test --- .../java/org/apache/spark/launcher/SparkLauncherSuite.java | 2 +- .../java/org/apache/spark/launcher/ChildThreadAppHandle.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index e097eff0c97d..8ed3742ee704 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -211,7 +211,7 @@ public static class SparkLauncherTestApp { public static void main(String[] args) throws Exception { assertEquals(1, args.length); - if("proc".equalsIgnoreCase(args[0])) { + if ("proc".equalsIgnoreCase(args[0])) { assertEquals("proc", args[0]); assertEquals("bar", System.getProperty("foo")); } else { diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java index c44ed96e28fa..792b33774060 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java @@ -53,8 +53,8 @@ void setChildThread(Thread childThread) { } protected void waitFor() throws InterruptedException { - if(this.childThread.isAlive()) { - this.childThread.wait(); + if (this.childThread.isAlive()) { + this.childThread.join(); } } } From 10513ec4a1cee2de26104945dc2eae54efeed9d2 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 14 Jun 2017 15:29:43 -0400 Subject: [PATCH 50/50] Fixing review comments --- .../java/org/apache/spark/launcher/AbstractSparkAppHandle.java | 2 +- .../java/org/apache/spark/launcher/ChildProcAppHandle.java | 2 +- .../java/org/apache/spark/launcher/ChildThreadAppHandle.java | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java index 182c723cbff8..5e25d7553ec4 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractSparkAppHandle.java @@ -130,5 +130,5 @@ private synchronized void fireEvent(boolean isInfoChanged) { } } - protected abstract void waitFor() throws InterruptedException; + abstract void waitFor() throws InterruptedException; } diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java index 1e350f23a400..5f2fd946815b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java @@ -58,7 +58,7 @@ void setChildProc(Process childProc, String loggerName) { SparkLauncher.REDIRECTOR_FACTORY); } - protected void waitFor() throws InterruptedException { + void waitFor() throws InterruptedException { this.childProc.waitFor(); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java index 792b33774060..79d61e67ab04 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildThreadAppHandle.java @@ -52,7 +52,8 @@ void setChildThread(Thread childThread) { this.childThread = childThread; } - protected void waitFor() throws InterruptedException { + @Override + void waitFor() throws InterruptedException { if (this.childThread.isAlive()) { this.childThread.join(); }