From daa5b70d66b32d582dc7c2cdba79ce748ca5cc66 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 30 Jan 2018 11:35:54 -0800 Subject: [PATCH 1/3] [SPARK-23020][core] Fix another race in the in-process launcher test. First the bad news: there's an unfixable race in the launcher code. (By unfixable I mean it would take a lot more effort than this change to fix it.) The good news is that it should only affect super short lived applications, such as the one run by the flaky test, so it's possible to work around it in our test. The fix also uncovered an issue with the recently added "closeAndWait()" method; closing the connection would still possibly cause data loss, so this change waits a while for the connection to finish itself, and closes the socket if that times out. The existing connection timeout is reused so that if desired it's possible to control how long to wait. As part of that I also restored the old behavior that disconnect() would force a disconnection from the child app; the "wait for data to arrive" approach is only taken when disposing of the handle. I tested this by inserting a bunch of sleeps in the test and the socket handling code in the launcher library; with those I was able to reproduce the error from the jenkins jobs. With the changes, even with all the sleeps still in place, all tests pass. --- .../spark/launcher/SparkLauncherSuite.java | 40 ++++++++++++++--- .../spark/launcher/AbstractAppHandle.java | 43 ++++++++++++------- .../spark/launcher/ChildProcAppHandle.java | 2 +- .../spark/launcher/InProcessAppHandle.java | 2 +- .../apache/spark/launcher/LauncherServer.java | 28 ++++++++---- .../spark/launcher/LauncherServerSuite.java | 2 +- 6 files changed, 83 insertions(+), 34 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 1543f4fdb0162..2225591a4ff75 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -157,12 +157,24 @@ private void inProcessLauncherTestImpl() throws Exception { SparkAppHandle handle = null; try { - handle = new InProcessLauncher() - .setMaster("local") - .setAppResource(SparkLauncher.NO_RESOURCE) - .setMainClass(InProcessTestApp.class.getName()) - .addAppArgs("hello") - .startApplication(listener); + synchronized (InProcessTestApp.LOCK) { + handle = new InProcessLauncher() + .setMaster("local") + .setAppResource(SparkLauncher.NO_RESOURCE) + .setMainClass(InProcessTestApp.class.getName()) + .addAppArgs("hello") + .startApplication(listener); + + // SPARK-23020: see doc for InProcessTestApp.LOCK for a description of the race. Here + // we wait until we know that the connection between the app and the launcher has been + // established before allowing the app to finish. + final SparkAppHandle _handle = handle; + eventually(Duration.ofSeconds(5), Duration.ofMillis(10), () -> { + assertNotEquals(SparkAppHandle.State.UNKNOWN, _handle.getState()); + }); + + InProcessTestApp.LOCK.wait(5000); + } waitFor(handle); assertEquals(SparkAppHandle.State.FINISHED, handle.getState()); @@ -193,10 +205,26 @@ public static void main(String[] args) throws Exception { public static class InProcessTestApp { + /** + * SPARK-23020: there's a race caused by a child app finishing too quickly. This would cause + * the InProcessAppHandle to dispose of itself even before the child connection was properly + * established, so no state changes would be detected for the application and its final + * state would be LOST. + * + * It's not really possible to fix that race safely in the handle code itself without changing + * the way in-process apps talk to the launcher library, so we work around that in the test by + * synchronizing on this object. + */ + public static final Object LOCK = new Object(); + public static void main(String[] args) throws Exception { assertNotEquals(0, args.length); assertEquals(args[0], "hello"); new SparkContext().stop(); + + synchronized (LOCK) { + LOCK.notifyAll(); + } } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java index 84a25a5254151..dcdb7e4200970 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java @@ -18,22 +18,22 @@ package org.apache.spark.launcher; import java.io.IOException; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicReference; import java.util.logging.Level; import java.util.logging.Logger; abstract class AbstractAppHandle implements SparkAppHandle { - private static final Logger LOG = Logger.getLogger(ChildProcAppHandle.class.getName()); + private static final Logger LOG = Logger.getLogger(AbstractAppHandle.class.getName()); private final LauncherServer server; private LauncherServer.ServerConnection connection; private List listeners; private AtomicReference state; - private String appId; + private volatile String appId; private volatile boolean disposed; protected AbstractAppHandle(LauncherServer server) { @@ -42,9 +42,9 @@ protected AbstractAppHandle(LauncherServer server) { } @Override - public synchronized void addListener(Listener l) { + public void addListener(Listener l) { if (listeners == null) { - listeners = new ArrayList<>(); + listeners = new CopyOnWriteArrayList<>(); } listeners.add(l); } @@ -71,16 +71,14 @@ public void stop() { @Override public synchronized void disconnect() { - if (!isDisposed()) { - if (connection != null) { - try { - connection.closeAndWait(); - } catch (IOException ioe) { - // no-op. - } + if (connection != null && connection.isOpen()) { + try { + connection.close(); + } catch (IOException ioe) { + // no-op. } - dispose(); } + dispose(); } void setConnection(LauncherServer.ServerConnection connection) { @@ -100,7 +98,18 @@ boolean isDisposed() { */ synchronized void dispose() { if (!isDisposed()) { + // First wait for all data from the connection to be read. Then unregister the handle. + // Otherwise, unregistering might cause the server to be stopped and all child connections + // to be closed. + if (connection != null) { + try { + connection.waitForClose(); + } catch (IOException ioe) { + // no-op. + } + } server.unregister(this); + // Set state to LOST if not yet final. setState(State.LOST, false); this.disposed = true; @@ -127,11 +136,13 @@ void setState(State s, boolean force) { current = state.get(); } - LOG.log(Level.WARNING, "Backend requested transition from final state {0} to {1}.", - new Object[] { current, s }); + if (s != State.LOST) { + LOG.log(Level.WARNING, "Backend requested transition from final state {0} to {1}.", + new Object[] { current, s }); + } } - synchronized void setAppId(String appId) { + void setAppId(String appId) { this.appId = appId; fireEvent(true); } 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 5e3c95676ecbe..5609f8492f4f4 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java @@ -112,7 +112,7 @@ void monitorChild() { } } - disconnect(); + dispose(); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java index b8030e0063a37..4b740d3fad20e 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java @@ -66,7 +66,7 @@ synchronized void start(String appName, Method main, String[] args) { setState(State.FAILED); } - disconnect(); + dispose(); }); app.setName(String.format(THREAD_NAME_FMT, THREAD_IDS.incrementAndGet(), appName)); 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 f4ecd52fdeab8..820c6b2d58a4d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -238,6 +238,7 @@ public void run() { }; ServerConnection clientConnection = new ServerConnection(client, timeout); Thread clientThread = factory.newThread(clientConnection); + clientConnection.setConnectionThread(clientThread); synchronized (clients) { clients.add(clientConnection); } @@ -290,17 +291,15 @@ class ServerConnection extends LauncherConnection { private TimerTask timeout; private volatile Thread connectionThread; - volatile AbstractAppHandle handle; + private volatile AbstractAppHandle handle; ServerConnection(Socket socket, TimerTask timeout) throws IOException { super(socket); this.timeout = timeout; } - @Override - public void run() { - this.connectionThread = Thread.currentThread(); - super.run(); + void setConnectionThread(Thread t) { + this.connectionThread = t; } @Override @@ -363,17 +362,28 @@ public void close() throws IOException { /** * Close the connection and wait for any buffered data to be processed before returning. * This ensures any changes reported by the child application take effect. + * + * This method allows a short period for the connection thread to finish by itself (same amount + * of time as the connection timeout, which is configurable). This should be fine for + * well-behaved applications, where they close the connection when the app handle detects the + * app has finished. + * + * In case the connection is not closed within the grace period, this method forcefully closes + * it and any subsequent data that may arrive will be ignored. */ - public void closeAndWait() throws IOException { - close(); - + public void waitForClose() throws IOException { Thread connThread = this.connectionThread; if (Thread.currentThread() != connThread) { try { - connThread.join(); + connThread.join(getConnectionTimeout()); } catch (InterruptedException ie) { // Ignore. } + + if (connThread.isAlive()) { + LOG.log(Level.WARNING, "Timed out waiting for child connection to close."); + close(); + } } } diff --git a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java index 024efac33c391..d16337a319be3 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java @@ -94,8 +94,8 @@ public void infoChanged(SparkAppHandle handle) { Message stopMsg = client.inbound.poll(30, TimeUnit.SECONDS); assertTrue(stopMsg instanceof Stop); } finally { - handle.kill(); close(client); + handle.kill(); client.clientThread.join(); } } From b96777573bdc9dc92b3419fb44bbd790117ee00e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 31 Jan 2018 18:32:38 -0800 Subject: [PATCH 2/3] Feedback. --- .../java/org/apache/spark/launcher/AbstractAppHandle.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java index dcdb7e4200970..9cbebdaeb33d3 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java @@ -42,7 +42,7 @@ protected AbstractAppHandle(LauncherServer server) { } @Override - public void addListener(Listener l) { + public synchronized void addListener(Listener l) { if (listeners == null) { listeners = new CopyOnWriteArrayList<>(); } @@ -95,6 +95,10 @@ boolean isDisposed() { /** * Mark the handle as disposed, and set it as LOST in case the current state is not final. + * + * This method should be called only when there's a reasonable expectation that the communication + * with the child application is not needed anymore, either because the code managing the handle + * has said so, or because the child application is finished. */ synchronized void dispose() { if (!isDisposed()) { From 82c276fa011588448e4c2ec122aac06d0fb02bb6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 1 Feb 2018 11:33:07 -0800 Subject: [PATCH 3/3] Reword comment. --- .../java/org/apache/spark/launcher/LauncherServer.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 820c6b2d58a4d..607879fd02ea9 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -360,12 +360,12 @@ public void close() throws IOException { } /** - * Close the connection and wait for any buffered data to be processed before returning. + * Wait for the remote side to close the connection so that any pending data is processed. * This ensures any changes reported by the child application take effect. * - * This method allows a short period for the connection thread to finish by itself (same amount - * of time as the connection timeout, which is configurable). This should be fine for - * well-behaved applications, where they close the connection when the app handle detects the + * This method allows a short period for the above to happen (same amount of time as the + * connection timeout, which is configurable). This should be fine for well-behaved + * applications, where they close the connection arond the same time the app handle detects the * app has finished. * * In case the connection is not closed within the grace period, this method forcefully closes