Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[rpc] Fix detecting bind failure in case of Netty EPOLL transport #287

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.alibaba.fluss.shaded.netty4.io.netty.channel.socket.SocketChannel;
import com.alibaba.fluss.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
import com.alibaba.fluss.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
import com.alibaba.fluss.shaded.netty4.io.netty.channel.unix.Errors;
import com.alibaba.fluss.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory;

import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -101,4 +102,19 @@ public static CompletableFuture<Void> shutdownChannel(Channel channel) {
}
return shutdownFuture;
}

/**
* check whether the provided {@link Throwable} represents a bind failure.
*
* @param t The {@link Throwable} object to be checked for bind failure.
* @return {@code true} if the provided {@link Throwable} represents a bind failure, {@code
* false} otherwise.
*/
public static boolean isBindFailure(Throwable t) {
return t instanceof java.net.BindException
|| (t instanceof Errors.NativeIoException
&& t.getMessage() != null
&& t.getMessage().matches("^bind\\(.*\\) failed:.*"))
|| (t.getCause() != null && isBindFailure(t.getCause()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledExecutorService;

import static com.alibaba.fluss.rpc.netty.NettyUtils.isBindFailure;
import static com.alibaba.fluss.rpc.netty.NettyUtils.shutdownChannel;
import static com.alibaba.fluss.rpc.netty.NettyUtils.shutdownGroup;
import static com.alibaba.fluss.utils.Preconditions.checkNotNull;
Expand Down Expand Up @@ -102,7 +103,7 @@ public void start() throws IOException {

this.acceptorGroup =
NettyUtils.newEventLoopGroup(
1, // always use single thread for accepter
1, // always use single thread for acceptor
"fluss-netty-server-acceptor");
this.selectorGroup =
NettyUtils.newEventLoopGroup(numNetworkThreads, "fluss-netty-server-selector");
Expand Down Expand Up @@ -139,11 +140,12 @@ public void start() throws IOException {
try {
bindChannel = bootstrap.bind().syncUninterruptibly().channel();
} catch (Exception e) {
LOG.debug("Failed to bind Netty server on port {}: {}", port, e.getMessage());
// syncUninterruptibly() throws checked exceptions via Unsafe
// continue if the exception is due to the port being in use, fail early
// otherwise
if (!(e instanceof BindException)) {
if (isBindFailure(e)) {
LOG.debug("Failed to bind Netty server on port {}: {}", port, e.getMessage());
} else {
throw e;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import com.alibaba.fluss.rpc.messages.LookupRequest;
import com.alibaba.fluss.rpc.messages.PbLookupReqForBucket;
import com.alibaba.fluss.rpc.metrics.TestingClientMetricGroup;
import com.alibaba.fluss.rpc.netty.NettyUtils;
import com.alibaba.fluss.rpc.netty.server.NettyServer;
import com.alibaba.fluss.rpc.netty.server.RequestsMetrics;
import com.alibaba.fluss.rpc.protocol.ApiKeys;
Expand Down Expand Up @@ -164,6 +165,21 @@ void testServerDisconnection() throws Exception {
.isEqualTo(serverNode);
}

@Test
void testBindFailureDetection() {
Throwable ex = new java.net.BindException();
assertThat(NettyUtils.isBindFailure(ex)).isTrue();

ex = new Exception(new java.net.BindException());
assertThat(NettyUtils.isBindFailure(ex)).isTrue();

ex = new Exception();
assertThat(NettyUtils.isBindFailure(ex)).isFalse();

ex = new RuntimeException();
assertThat(NettyUtils.isBindFailure(ex)).isFalse();
}

private void buildNettyServer(int serverId) throws Exception {
try (NetUtils.Port availablePort = getAvailablePort()) {
serverNode =
Expand Down