Skip to content

Commit 7b87283

Browse files
committed
fix
1 parent 68aed9d commit 7b87283

File tree

2 files changed

+17
-4
lines changed

2 files changed

+17
-4
lines changed

ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java

Lines changed: 16 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,16 @@
4444
* IO related utility methods.
4545
*/
4646
public interface IOUtils {
47+
// Network exceptions for reconnection
48+
Class<?>[] NETWORK_EXCEPTIONS = {
49+
SocketException.class,
50+
SocketTimeoutException.class,
51+
ClosedChannelException.class,
52+
EOFException.class,
53+
AlreadyClosedException.class,
54+
TimeoutIOException.class
55+
};
56+
4757
static InterruptedIOException toInterruptedIOException(
4858
String message, InterruptedException e) {
4959
final InterruptedIOException iioe = new InterruptedIOException(message);
@@ -91,9 +101,12 @@ static <T> T getFromFuture(CompletableFuture<T> future, Supplier<Object> name, T
91101
}
92102

93103
static boolean shouldReconnect(Throwable e) {
94-
return ReflectionUtils.isInstance(e,
95-
SocketException.class, SocketTimeoutException.class, ClosedChannelException.class, EOFException.class,
96-
AlreadyClosedException.class, TimeoutIOException.class);
104+
if (e == null) return false;
105+
if (ReflectionUtils.isInstance(e, NETWORK_EXCEPTIONS)) {
106+
return true;
107+
}
108+
return e.getCause() != null &&
109+
ReflectionUtils.isInstance(e.getCause(), NETWORK_EXCEPTIONS);
97110
}
98111

99112
static void readFully(InputStream in, int buffSize) throws IOException {

ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ protected void channelRead0(ChannelHandlerContext ctx,
128128

129129
@Override
130130
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
131-
failOutstandingRequests(cause);
131+
failOutstandingRequests(new IOException("Caught an exception for the connection to " + peer, cause));
132132
client.close();
133133
}
134134

0 commit comments

Comments
 (0)