Skip to content

Commit 1c4d10b

Browse files
JoshRosenliancheng
authored andcommitted
[SPARK-19529] TransportClientFactory.createClient() shouldn't call awaitUninterruptibly()
## 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 <[email protected]> Closes #16866 from JoshRosen/SPARK-19529.
1 parent ab88b24 commit 1c4d10b

File tree

9 files changed

+30
-24
lines changed

9 files changed

+30
-24
lines changed

common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -122,7 +122,8 @@ public TransportClientFactory(
122122
*
123123
* Concurrency: This method is safe to call from multiple threads.
124124
*/
125-
public TransportClient createClient(String remoteHost, int remotePort) throws IOException {
125+
public TransportClient createClient(String remoteHost, int remotePort)
126+
throws IOException, InterruptedException {
126127
// Get connection from the connection pool first.
127128
// If it is not found or not active, create a new one.
128129
// Use unresolved address here to avoid DNS resolution each time we creates a client.
@@ -190,13 +191,14 @@ public TransportClient createClient(String remoteHost, int remotePort) throws IO
190191
* As with {@link #createClient(String, int)}, this method is blocking.
191192
*/
192193
public TransportClient createUnmanagedClient(String remoteHost, int remotePort)
193-
throws IOException {
194+
throws IOException, InterruptedException {
194195
final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
195196
return createClient(address);
196197
}
197198

198199
/** Create a completely new {@link TransportClient} to the remote address. */
199-
private TransportClient createClient(InetSocketAddress address) throws IOException {
200+
private TransportClient createClient(InetSocketAddress address)
201+
throws IOException, InterruptedException {
200202
logger.debug("Creating new connection to {}", address);
201203

202204
Bootstrap bootstrap = new Bootstrap();
@@ -223,7 +225,7 @@ public void initChannel(SocketChannel ch) {
223225
// Connect to the remote server
224226
long preConnect = System.nanoTime();
225227
ChannelFuture cf = bootstrap.connect(address);
226-
if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) {
228+
if (!cf.await(conf.connectionTimeoutMs())) {
227229
throw new IOException(
228230
String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs()));
229231
} else if (cf.cause() != null) {

common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -99,6 +99,8 @@ public void run() {
9999
clients.add(client);
100100
} catch (IOException e) {
101101
failed.incrementAndGet();
102+
} catch (InterruptedException e) {
103+
throw new RuntimeException(e);
102104
}
103105
}
104106
};
@@ -142,7 +144,7 @@ public void reuseClientsUpToConfigVariableConcurrent() throws Exception {
142144
}
143145

144146
@Test
145-
public void returnDifferentClientsForDifferentServers() throws IOException {
147+
public void returnDifferentClientsForDifferentServers() throws IOException, InterruptedException {
146148
TransportClientFactory factory = context.createClientFactory();
147149
TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort());
148150
TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort());
@@ -171,7 +173,7 @@ public void neverReturnInactiveClients() throws IOException, InterruptedExceptio
171173
}
172174

173175
@Test
174-
public void closeBlockClientsWithFactory() throws IOException {
176+
public void closeBlockClientsWithFactory() throws IOException, InterruptedException {
175177
TransportClientFactory factory = context.createClientFactory();
176178
TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort());
177179
TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort());

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -94,7 +94,7 @@ public void fetchBlocks(
9494
new RetryingBlockFetcher.BlockFetchStarter() {
9595
@Override
9696
public void createAndStart(String[] blockIds, BlockFetchingListener listener)
97-
throws IOException {
97+
throws IOException, InterruptedException {
9898
TransportClient client = clientFactory.createClient(host, port);
9999
new OneForOneBlockFetcher(client, appId, execId, blockIds, listener).start();
100100
}
@@ -129,7 +129,7 @@ public void registerWithShuffleServer(
129129
String host,
130130
int port,
131131
String execId,
132-
ExecutorShuffleInfo executorInfo) throws IOException {
132+
ExecutorShuffleInfo executorInfo) throws IOException, InterruptedException {
133133
checkInit();
134134
TransportClient client = clientFactory.createUnmanagedClient(host, port);
135135
try {

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -57,7 +57,8 @@ public interface BlockFetchStarter {
5757
* {@link org.apache.spark.network.client.TransportClientFactory} in order to fix connection
5858
* issues.
5959
*/
60-
void createAndStart(String[] blockIds, BlockFetchingListener listener) throws IOException;
60+
void createAndStart(String[] blockIds, BlockFetchingListener listener)
61+
throws IOException, InterruptedException;
6162
}
6263

6364
/** Shared executor service used for waiting and retrying. */

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -68,7 +68,7 @@ public void registerDriverWithShuffleService(
6868
String host,
6969
int port,
7070
long heartbeatTimeoutMs,
71-
long heartbeatIntervalMs) throws IOException {
71+
long heartbeatIntervalMs) throws IOException, InterruptedException {
7272

7373
checkInit();
7474
ByteBuffer registerDriver = new RegisterDriver(appId, heartbeatTimeoutMs).toByteBuffer();

common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -103,7 +103,7 @@ public void afterEach() {
103103
}
104104

105105
@Test
106-
public void testGoodClient() throws IOException {
106+
public void testGoodClient() throws IOException, InterruptedException {
107107
clientFactory = context.createClientFactory(
108108
Lists.<TransportClientBootstrap>newArrayList(
109109
new SaslClientBootstrap(conf, "app-1", secretKeyHolder)));
@@ -133,7 +133,7 @@ public void testBadClient() {
133133
}
134134

135135
@Test
136-
public void testNoSaslClient() throws IOException {
136+
public void testNoSaslClient() throws IOException, InterruptedException {
137137
clientFactory = context.createClientFactory(
138138
Lists.<TransportClientBootstrap>newArrayList());
139139

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -242,7 +242,7 @@ public void testFetchNoServer() throws Exception {
242242
}
243243

244244
private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo)
245-
throws IOException {
245+
throws IOException, InterruptedException {
246246
ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false);
247247
client.init(APP_ID);
248248
client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(),

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -60,7 +60,7 @@ public void afterEach() {
6060
}
6161

6262
@Test
63-
public void testValid() throws IOException {
63+
public void testValid() throws IOException, InterruptedException {
6464
validate("my-app-id", "secret", false);
6565
}
6666

@@ -83,12 +83,13 @@ public void testBadSecret() {
8383
}
8484

8585
@Test
86-
public void testEncryption() throws IOException {
86+
public void testEncryption() throws IOException, InterruptedException {
8787
validate("my-app-id", "secret", true);
8888
}
8989

9090
/** Creates an ExternalShuffleClient and attempts to register with the server. */
91-
private void validate(String appId, String secretKey, boolean encrypt) throws IOException {
91+
private void validate(String appId, String secretKey, boolean encrypt)
92+
throws IOException, InterruptedException {
9293
TransportConf testConf = conf;
9394
if (encrypt) {
9495
testConf = new TransportConf("shuffle", new MapConfigProvider(

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,7 @@ public class RetryingBlockFetcherSuite {
5252
ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19]));
5353

5454
@Test
55-
public void testNoFailures() throws IOException {
55+
public void testNoFailures() throws IOException, InterruptedException {
5656
BlockFetchingListener listener = mock(BlockFetchingListener.class);
5757

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

7373
@Test
74-
public void testUnrecoverableFailure() throws IOException {
74+
public void testUnrecoverableFailure() throws IOException, InterruptedException {
7575
BlockFetchingListener listener = mock(BlockFetchingListener.class);
7676

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

9292
@Test
93-
public void testSingleIOExceptionOnFirst() throws IOException {
93+
public void testSingleIOExceptionOnFirst() throws IOException, InterruptedException {
9494
BlockFetchingListener listener = mock(BlockFetchingListener.class);
9595

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

115115
@Test
116-
public void testSingleIOExceptionOnSecond() throws IOException {
116+
public void testSingleIOExceptionOnSecond() throws IOException, InterruptedException {
117117
BlockFetchingListener listener = mock(BlockFetchingListener.class);
118118

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

137137
@Test
138-
public void testTwoIOExceptions() throws IOException {
138+
public void testTwoIOExceptions() throws IOException, InterruptedException {
139139
BlockFetchingListener listener = mock(BlockFetchingListener.class);
140140

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

165165
@Test
166-
public void testThreeIOExceptions() throws IOException {
166+
public void testThreeIOExceptions() throws IOException, InterruptedException {
167167
BlockFetchingListener listener = mock(BlockFetchingListener.class);
168168

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

197197
@Test
198-
public void testRetryAndUnrecoverable() throws IOException {
198+
public void testRetryAndUnrecoverable() throws IOException, InterruptedException {
199199
BlockFetchingListener listener = mock(BlockFetchingListener.class);
200200

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

243243
MapConfigProvider provider = new MapConfigProvider(ImmutableMap.of(
244244
"spark.shuffle.io.maxRetries", "2",

0 commit comments

Comments
 (0)