Skip to content

Commit

Permalink
Stop Spark Application if launcher goes down and use reflection
Browse files Browse the repository at this point in the history
  • Loading branch information
kishorvpatil committed Sep 8, 2016
1 parent b230fb9 commit 8986aa9
Show file tree
Hide file tree
Showing 10 changed files with 434 additions and 139 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand All @@ -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 {
Expand All @@ -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()
}
}
}

Expand Down Expand Up @@ -115,7 +157,6 @@ private[spark] abstract class LauncherBackend {
_isConnected = false
}
}

}

}
Expand Down
Original file line number Diff line number Diff line change
@@ -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<Listener> 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);
}
}
}
}

}

Original file line number Diff line number Diff line change
Expand Up @@ -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<Listener> 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
Expand All @@ -103,67 +46,22 @@ 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;
}
}
}

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);
}
}
}
}

}
Loading

0 comments on commit 8986aa9

Please sign in to comment.