Skip to content

Commit

Permalink
[SPARK-19529] TransportClientFactory.createClient() shouldn't call aw…
Browse files Browse the repository at this point in the history
…aitUninterruptibly()

## What changes were proposed in this pull request?

This patch replaces a single `awaitUninterruptibly()` call with a plain `await()` call in Spark's `network-common` library in order to fix a bug which may cause tasks to be uncancellable.

In Spark's Netty RPC layer, `TransportClientFactory.createClient()` calls `awaitUninterruptibly()` on a Netty future while waiting for a connection to be established. This creates problem when a Spark task is interrupted while blocking in this call (which can happen in the event of a slow connection which will eventually time out). This has bad impacts on task cancellation when `interruptOnCancel = true`.

As an example of the impact of this problem, I experienced significant numbers of uncancellable "zombie tasks" on a production cluster where several tasks were blocked trying to connect to a dead shuffle server and then continued running as zombies after I cancelled the associated Spark stage. The zombie tasks ran for several minutes with the following stack:

```
java.lang.Object.wait(Native Method)
java.lang.Object.wait(Object.java:460)
io.netty.util.concurrent.DefaultPromise.await0(DefaultPromise.java:607)
io.netty.util.concurrent.DefaultPromise.awaitUninterruptibly(DefaultPromise.java:301)
org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:224)
org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:179) => holding Monitor(java.lang.Object1849476028})
org.apache.spark.network.shuffle.ExternalShuffleClient$1.createAndStart(ExternalShuffleClient.java:105)
org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:140)
org.apache.spark.network.shuffle.RetryingBlockFetcher.start(RetryingBlockFetcher.java:120)
org.apache.spark.network.shuffle.ExternalShuffleClient.fetchBlocks(ExternalShuffleClient.java:114)
org.apache.spark.storage.ShuffleBlockFetcherIterator.sendRequest(ShuffleBlockFetcherIterator.scala:169)
org.apache.spark.storage.ShuffleBlockFetcherIterator.fetchUpToMaxBytes(ShuffleBlockFetcherIterator.scala:
350)
org.apache.spark.storage.ShuffleBlockFetcherIterator.initialize(ShuffleBlockFetcherIterator.scala:286)
org.apache.spark.storage.ShuffleBlockFetcherIterator.<init>(ShuffleBlockFetcherIterator.scala:120)
org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:45)
org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:169)
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
[...]
```

As far as I can tell, `awaitUninterruptibly()` might have been used in order to avoid having to declare that methods throw `InterruptedException` (this code is written in Java, hence the need to use checked exceptions). This patch simply replaces this with a regular, interruptible `await()` call,.

This required several interface changes to declare a new checked exception (these are internal interfaces, though, and this change doesn't significantly impact binary compatibility).

An alternative approach would be to wrap `InterruptedException` into `IOException` in order to avoid having to change interfaces. The problem with this approach is that the `network-shuffle` project's `RetryingBlockFetcher` code treats `IOExceptions` as transitive failures when deciding whether to retry fetches, so throwing a wrapped `IOException` might cause an interrupted shuffle fetch to be retried, further prolonging the lifetime of a cancelled zombie task.

Note that there are three other `awaitUninterruptibly()` in the codebase, but those calls have a hard 10 second timeout and are waiting on a `close()` operation which is expected to complete near instantaneously, so the impact of uninterruptibility there is much smaller.

## How was this patch tested?

Manually.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #16866 from JoshRosen/SPARK-19529.
  • Loading branch information
JoshRosen authored and liancheng committed Feb 13, 2017
1 parent ab88b24 commit 1c4d10b
Show file tree
Hide file tree
Showing 9 changed files with 30 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,8 @@ public TransportClientFactory(
*
* Concurrency: This method is safe to call from multiple threads.
*/
public TransportClient createClient(String remoteHost, int remotePort) throws IOException {
public TransportClient createClient(String remoteHost, int remotePort)
throws IOException, InterruptedException {
// Get connection from the connection pool first.
// If it is not found or not active, create a new one.
// Use unresolved address here to avoid DNS resolution each time we creates a client.
Expand Down Expand Up @@ -190,13 +191,14 @@ public TransportClient createClient(String remoteHost, int remotePort) throws IO
* As with {@link #createClient(String, int)}, this method is blocking.
*/
public TransportClient createUnmanagedClient(String remoteHost, int remotePort)
throws IOException {
throws IOException, InterruptedException {
final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
return createClient(address);
}

/** Create a completely new {@link TransportClient} to the remote address. */
private TransportClient createClient(InetSocketAddress address) throws IOException {
private TransportClient createClient(InetSocketAddress address)
throws IOException, InterruptedException {
logger.debug("Creating new connection to {}", address);

Bootstrap bootstrap = new Bootstrap();
Expand All @@ -223,7 +225,7 @@ public void initChannel(SocketChannel ch) {
// Connect to the remote server
long preConnect = System.nanoTime();
ChannelFuture cf = bootstrap.connect(address);
if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) {
if (!cf.await(conf.connectionTimeoutMs())) {
throw new IOException(
String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs()));
} else if (cf.cause() != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,8 @@ public void run() {
clients.add(client);
} catch (IOException e) {
failed.incrementAndGet();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
};
Expand Down Expand Up @@ -142,7 +144,7 @@ public void reuseClientsUpToConfigVariableConcurrent() throws Exception {
}

@Test
public void returnDifferentClientsForDifferentServers() throws IOException {
public void returnDifferentClientsForDifferentServers() throws IOException, InterruptedException {
TransportClientFactory factory = context.createClientFactory();
TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort());
TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort());
Expand Down Expand Up @@ -171,7 +173,7 @@ public void neverReturnInactiveClients() throws IOException, InterruptedExceptio
}

@Test
public void closeBlockClientsWithFactory() throws IOException {
public void closeBlockClientsWithFactory() throws IOException, InterruptedException {
TransportClientFactory factory = context.createClientFactory();
TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort());
TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ public void fetchBlocks(
new RetryingBlockFetcher.BlockFetchStarter() {
@Override
public void createAndStart(String[] blockIds, BlockFetchingListener listener)
throws IOException {
throws IOException, InterruptedException {
TransportClient client = clientFactory.createClient(host, port);
new OneForOneBlockFetcher(client, appId, execId, blockIds, listener).start();
}
Expand Down Expand Up @@ -129,7 +129,7 @@ public void registerWithShuffleServer(
String host,
int port,
String execId,
ExecutorShuffleInfo executorInfo) throws IOException {
ExecutorShuffleInfo executorInfo) throws IOException, InterruptedException {
checkInit();
TransportClient client = clientFactory.createUnmanagedClient(host, port);
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,8 @@ public interface BlockFetchStarter {
* {@link org.apache.spark.network.client.TransportClientFactory} in order to fix connection
* issues.
*/
void createAndStart(String[] blockIds, BlockFetchingListener listener) throws IOException;
void createAndStart(String[] blockIds, BlockFetchingListener listener)
throws IOException, InterruptedException;
}

/** Shared executor service used for waiting and retrying. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ public void registerDriverWithShuffleService(
String host,
int port,
long heartbeatTimeoutMs,
long heartbeatIntervalMs) throws IOException {
long heartbeatIntervalMs) throws IOException, InterruptedException {

checkInit();
ByteBuffer registerDriver = new RegisterDriver(appId, heartbeatTimeoutMs).toByteBuffer();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ public void afterEach() {
}

@Test
public void testGoodClient() throws IOException {
public void testGoodClient() throws IOException, InterruptedException {
clientFactory = context.createClientFactory(
Lists.<TransportClientBootstrap>newArrayList(
new SaslClientBootstrap(conf, "app-1", secretKeyHolder)));
Expand Down Expand Up @@ -133,7 +133,7 @@ public void testBadClient() {
}

@Test
public void testNoSaslClient() throws IOException {
public void testNoSaslClient() throws IOException, InterruptedException {
clientFactory = context.createClientFactory(
Lists.<TransportClientBootstrap>newArrayList());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,7 +242,7 @@ public void testFetchNoServer() throws Exception {
}

private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo)
throws IOException {
throws IOException, InterruptedException {
ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false);
client.init(APP_ID);
client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ public void afterEach() {
}

@Test
public void testValid() throws IOException {
public void testValid() throws IOException, InterruptedException {
validate("my-app-id", "secret", false);
}

Expand All @@ -83,12 +83,13 @@ public void testBadSecret() {
}

@Test
public void testEncryption() throws IOException {
public void testEncryption() throws IOException, InterruptedException {
validate("my-app-id", "secret", true);
}

/** Creates an ExternalShuffleClient and attempts to register with the server. */
private void validate(String appId, String secretKey, boolean encrypt) throws IOException {
private void validate(String appId, String secretKey, boolean encrypt)
throws IOException, InterruptedException {
TransportConf testConf = conf;
if (encrypt) {
testConf = new TransportConf("shuffle", new MapConfigProvider(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public class RetryingBlockFetcherSuite {
ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19]));

@Test
public void testNoFailures() throws IOException {
public void testNoFailures() throws IOException, InterruptedException {
BlockFetchingListener listener = mock(BlockFetchingListener.class);

List<? extends Map<String, Object>> interactions = Arrays.asList(
Expand All @@ -71,7 +71,7 @@ public void testNoFailures() throws IOException {
}

@Test
public void testUnrecoverableFailure() throws IOException {
public void testUnrecoverableFailure() throws IOException, InterruptedException {
BlockFetchingListener listener = mock(BlockFetchingListener.class);

List<? extends Map<String, Object>> interactions = Arrays.asList(
Expand All @@ -90,7 +90,7 @@ public void testUnrecoverableFailure() throws IOException {
}

@Test
public void testSingleIOExceptionOnFirst() throws IOException {
public void testSingleIOExceptionOnFirst() throws IOException, InterruptedException {
BlockFetchingListener listener = mock(BlockFetchingListener.class);

List<? extends Map<String, Object>> interactions = Arrays.asList(
Expand All @@ -113,7 +113,7 @@ public void testSingleIOExceptionOnFirst() throws IOException {
}

@Test
public void testSingleIOExceptionOnSecond() throws IOException {
public void testSingleIOExceptionOnSecond() throws IOException, InterruptedException {
BlockFetchingListener listener = mock(BlockFetchingListener.class);

List<? extends Map<String, Object>> interactions = Arrays.asList(
Expand All @@ -135,7 +135,7 @@ public void testSingleIOExceptionOnSecond() throws IOException {
}

@Test
public void testTwoIOExceptions() throws IOException {
public void testTwoIOExceptions() throws IOException, InterruptedException {
BlockFetchingListener listener = mock(BlockFetchingListener.class);

List<? extends Map<String, Object>> interactions = Arrays.asList(
Expand Down Expand Up @@ -163,7 +163,7 @@ public void testTwoIOExceptions() throws IOException {
}

@Test
public void testThreeIOExceptions() throws IOException {
public void testThreeIOExceptions() throws IOException, InterruptedException {
BlockFetchingListener listener = mock(BlockFetchingListener.class);

List<? extends Map<String, Object>> interactions = Arrays.asList(
Expand Down Expand Up @@ -195,7 +195,7 @@ public void testThreeIOExceptions() throws IOException {
}

@Test
public void testRetryAndUnrecoverable() throws IOException {
public void testRetryAndUnrecoverable() throws IOException, InterruptedException {
BlockFetchingListener listener = mock(BlockFetchingListener.class);

List<? extends Map<String, Object>> interactions = Arrays.asList(
Expand Down Expand Up @@ -238,7 +238,7 @@ public void testRetryAndUnrecoverable() throws IOException {
@SuppressWarnings("unchecked")
private static void performInteractions(List<? extends Map<String, Object>> interactions,
BlockFetchingListener listener)
throws IOException {
throws IOException, InterruptedException {

MapConfigProvider provider = new MapConfigProvider(ImmutableMap.of(
"spark.shuffle.io.maxRetries", "2",
Expand Down

0 comments on commit 1c4d10b

Please sign in to comment.