Skip to content

Commit b93e102

Browse files
committed
Merge branch 'SPARK-21253' into 'spark_2.1'
[SPARK-21253] 修复因为网络异常导致的下载依赖卡死的问题 修复因为网络异常导致的下载依赖卡死的问题 resolve apache#117 See merge request !68
2 parents 274042a + 9674814 commit b93e102

File tree

3 files changed

+59
-12
lines changed

3 files changed

+59
-12
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -189,7 +189,7 @@ public void stream(final String streamId, final StreamCallback callback) {
189189
// written to the socket atomically, so that callbacks are called in the right order
190190
// when responses arrive.
191191
synchronized (this) {
192-
handler.addStreamCallback(callback);
192+
handler.addStreamCallback(streamId, callback);
193193
channel.writeAndFlush(new StreamRequest(streamId)).addListener(
194194
new ChannelFutureListener() {
195195
@Override

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

Lines changed: 29 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,8 @@
2626

2727
import com.google.common.annotations.VisibleForTesting;
2828
import io.netty.channel.Channel;
29+
import org.apache.commons.lang3.tuple.ImmutablePair;
30+
import org.apache.commons.lang3.tuple.Pair;
2931
import org.slf4j.Logger;
3032
import org.slf4j.LoggerFactory;
3133

@@ -56,7 +58,7 @@ public class TransportResponseHandler extends MessageHandler<ResponseMessage> {
5658

5759
private final Map<Long, RpcResponseCallback> outstandingRpcs;
5860

59-
private final Queue<StreamCallback> streamCallbacks;
61+
private final Queue<Pair<String, StreamCallback>> streamCallbacks;
6062
private volatile boolean streamActive;
6163

6264
/** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */
@@ -88,9 +90,9 @@ public void removeRpcRequest(long requestId) {
8890
outstandingRpcs.remove(requestId);
8991
}
9092

91-
public void addStreamCallback(StreamCallback callback) {
93+
public void addStreamCallback(String streamId, StreamCallback callback) {
9294
timeOfLastRequestNs.set(System.nanoTime());
93-
streamCallbacks.offer(callback);
95+
streamCallbacks.offer(ImmutablePair.of(streamId, callback));
9496
}
9597

9698
@VisibleForTesting
@@ -104,15 +106,31 @@ public void deactivateStream() {
104106
*/
105107
private void failOutstandingRequests(Throwable cause) {
106108
for (Map.Entry<StreamChunkId, ChunkReceivedCallback> entry : outstandingFetches.entrySet()) {
107-
entry.getValue().onFailure(entry.getKey().chunkIndex, cause);
109+
try {
110+
entry.getValue().onFailure(entry.getKey().chunkIndex, cause);
111+
} catch (Exception e) {
112+
logger.warn("ChunkReceivedCallback.onFailure throws exception", e);
113+
}
108114
}
109115
for (Map.Entry<Long, RpcResponseCallback> entry : outstandingRpcs.entrySet()) {
110-
entry.getValue().onFailure(cause);
116+
try {
117+
entry.getValue().onFailure(cause);
118+
} catch (Exception e) {
119+
logger.warn("RpcResponseCallback.onFailure throws exception", e);
120+
}
121+
}
122+
for (Pair<String, StreamCallback> entry : streamCallbacks) {
123+
try {
124+
entry.getValue().onFailure(entry.getKey(), cause);
125+
} catch (Exception e) {
126+
logger.warn("StreamCallback.onFailure throws exception", e);
127+
}
111128
}
112129

113130
// It's OK if new fetches appear, as they will fail immediately.
114131
outstandingFetches.clear();
115132
outstandingRpcs.clear();
133+
streamCallbacks.clear();
116134
}
117135

118136
@Override
@@ -190,8 +208,9 @@ public void handle(ResponseMessage message) throws Exception {
190208
}
191209
} else if (message instanceof StreamResponse) {
192210
StreamResponse resp = (StreamResponse) message;
193-
StreamCallback callback = streamCallbacks.poll();
194-
if (callback != null) {
211+
Pair<String, StreamCallback> entry = streamCallbacks.poll();
212+
if (entry != null) {
213+
StreamCallback callback = entry.getValue();
195214
if (resp.byteCount > 0) {
196215
StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
197216
callback);
@@ -216,8 +235,9 @@ public void handle(ResponseMessage message) throws Exception {
216235
}
217236
} else if (message instanceof StreamFailure) {
218237
StreamFailure resp = (StreamFailure) message;
219-
StreamCallback callback = streamCallbacks.poll();
220-
if (callback != null) {
238+
Pair<String, StreamCallback> entry = streamCallbacks.poll();
239+
if (entry != null) {
240+
StreamCallback callback = entry.getValue();
221241
try {
222242
callback.onFailure(resp.streamId, new RuntimeException(resp.error));
223243
} catch (IOException ioe) {

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

Lines changed: 29 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.network;
1919

20+
import java.io.IOException;
2021
import java.nio.ByteBuffer;
2122

2223
import io.netty.channel.Channel;
@@ -130,17 +131,43 @@ public void testActiveStreams() throws Exception {
130131

131132
StreamResponse response = new StreamResponse("stream", 1234L, null);
132133
StreamCallback cb = mock(StreamCallback.class);
133-
handler.addStreamCallback(cb);
134+
handler.addStreamCallback("stream", cb);
134135
assertEquals(1, handler.numOutstandingRequests());
135136
handler.handle(response);
136137
assertEquals(1, handler.numOutstandingRequests());
137138
handler.deactivateStream();
138139
assertEquals(0, handler.numOutstandingRequests());
139140

140141
StreamFailure failure = new StreamFailure("stream", "uh-oh");
141-
handler.addStreamCallback(cb);
142+
handler.addStreamCallback("stream", cb);
142143
assertEquals(1, handler.numOutstandingRequests());
143144
handler.handle(failure);
144145
assertEquals(0, handler.numOutstandingRequests());
145146
}
147+
148+
@Test
149+
public void failOutstandingStreamCallbackOnClose() throws Exception {
150+
Channel c = new LocalChannel();
151+
c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder());
152+
TransportResponseHandler handler = new TransportResponseHandler(c);
153+
154+
StreamCallback cb = mock(StreamCallback.class);
155+
handler.addStreamCallback("stream-1", cb);
156+
handler.channelInactive();
157+
158+
verify(cb).onFailure(eq("stream-1"), isA(IOException.class));
159+
}
160+
161+
@Test
162+
public void failOutstandingStreamCallbackOnException() throws Exception {
163+
Channel c = new LocalChannel();
164+
c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder());
165+
TransportResponseHandler handler = new TransportResponseHandler(c);
166+
167+
StreamCallback cb = mock(StreamCallback.class);
168+
handler.addStreamCallback("stream-1", cb);
169+
handler.exceptionCaught(new IOException("Oops!"));
170+
171+
verify(cb).onFailure(eq("stream-1"), isA(IOException.class));
172+
}
146173
}

0 commit comments

Comments
 (0)