You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/05/20 13:32:22 UTC
flink git commit: [runtime] Improve error message for failed network
connections due to limited number of file handles
Repository: flink
Updated Branches:
refs/heads/master 825cea2df -> 89209a46d
[runtime] Improve error message for failed network connections due to limited number of file handles
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/89209a46
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/89209a46
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/89209a46
Branch: refs/heads/master
Commit: 89209a46d8329ebdb8c3bde7cda4061143eb0212
Parents: 825cea2
Author: Stephan Ewen <se...@apache.org>
Authored: Tue May 19 15:43:40 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed May 20 13:31:16 2015 +0200
----------------------------------------------------------------------
.../runtime/io/network/netty/NettyClient.java | 19 ++++++++++++++++++-
1 file changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/89209a46/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
index c24c743..201ec33 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
@@ -154,6 +154,23 @@ class NettyClient {
ChannelFuture connect(SocketAddress serverSocketAddress) {
checkState(bootstrap != null, "Client has not been initialized yet.");
- return bootstrap.connect(serverSocketAddress);
+ try {
+ return bootstrap.connect(serverSocketAddress);
+ }
+ catch (io.netty.channel.ChannelException e) {
+ if ( (e.getCause() instanceof java.net.SocketException &&
+ e.getCause().getMessage().equals("Too many open files")) ||
+ (e.getCause() instanceof io.netty.channel.ChannelException &&
+ e.getCause().getCause() instanceof java.net.SocketException &&
+ e.getCause().getCause().getMessage().equals("Too many open files")))
+ {
+ throw new io.netty.channel.ChannelException(
+ "The operating system does not offer enough file handles to open the network connection. " +
+ "Please increase the number of of available file handles.", e.getCause());
+ }
+ else {
+ throw e;
+ }
+ }
}
}