Skip to content

Commit

Permalink
Do not set SO_LINGER on server channels (#26997)
Browse files Browse the repository at this point in the history
Right now we are attempting to set SO_LINGER to 0 on server channels
when we are stopping the tcp transport. This is not a supported socket
option and throws an exception. This also prevents the channels from
being closed.

This commit 1. doesn't set SO_LINGER for server channges, 2. checks
that it is a supported option in nio, and 3. changes the log message
to warn for server channel close exceptions.
  • Loading branch information
Tim-Brooks authored Oct 13, 2017
1 parent bb013c6 commit 277637f
Show file tree
Hide file tree
Showing 5 changed files with 12 additions and 14 deletions.
10 changes: 4 additions & 6 deletions core/src/main/java/org/elasticsearch/transport/TcpTransport.java
Original file line number Diff line number Diff line change
Expand Up @@ -905,11 +905,9 @@ protected final void doStop() {
// first stop to accept any incoming connections so nobody can connect to this transport
for (Map.Entry<String, List<Channel>> entry : serverChannels.entrySet()) {
try {
closeChannels(entry.getValue(), true, true);
closeChannels(entry.getValue(), true, false);
} catch (Exception e) {
logger.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"Error closing serverChannel for profile [{}]", entry.getKey()), e);
logger.warn(new ParameterizedMessage("Error closing serverChannel for profile [{}]", entry.getKey()), e);
}
}
// we are holding a write lock so nobody modifies the connectedNodes / openConnections map - it's safe to first close
Expand Down Expand Up @@ -1024,9 +1022,9 @@ protected void innerOnFailure(Exception e) {
*
* @param channels the channels to close
* @param blocking whether the channels should be closed synchronously
* @param closingTransport whether we abort the connection on RST instead of FIN
* @param doNotLinger whether we abort the connection on RST instead of FIN
*/
protected abstract void closeChannels(List<Channel> channels, boolean blocking, boolean closingTransport) throws IOException;
protected abstract void closeChannels(List<Channel> channels, boolean blocking, boolean doNotLinger) throws IOException;

/**
* Sends message to channel. The listener's onResponse method will be called when the send is complete unless an exception
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,7 @@ protected Object bind(String name, InetSocketAddress address) throws IOException
}

@Override
protected void closeChannels(List channel, boolean blocking, boolean closingTransport) throws IOException {
protected void closeChannels(List channel, boolean blocking, boolean doNotLinger) throws IOException {

}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -331,8 +331,8 @@ protected void sendMessage(Channel channel, BytesReference reference, ActionList
}

@Override
protected void closeChannels(final List<Channel> channels, boolean blocking, boolean closingTransport) throws IOException {
if (closingTransport) {
protected void closeChannels(final List<Channel> channels, boolean blocking, boolean doNotLinger) throws IOException {
if (doNotLinger) {
for (Channel channel : channels) {
/* We set SO_LINGER timeout to 0 to ensure that when we shutdown the node we don't have a gazillion connections sitting
* in TIME_WAIT to free up resources quickly. This is really the only part where we close the connection from the server
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,8 +243,8 @@ protected void sendMessage(MockChannel mockChannel, BytesReference reference, Ac
}

@Override
protected void closeChannels(List<MockChannel> channels, boolean blocking, boolean closingTransport) throws IOException {
if (closingTransport) {
protected void closeChannels(List<MockChannel> channels, boolean blocking, boolean doNotLinger) throws IOException {
if (doNotLinger) {
for (MockChannel channel : channels) {
if (channel.activeChannel != null) {
/* We set SO_LINGER timeout to 0 to ensure that when we shutdown the node we don't have a gazillion connections sitting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,15 +99,15 @@ protected NioServerSocketChannel bind(String name, InetSocketAddress address) th
}

@Override
protected void closeChannels(List<NioChannel> channels, boolean blocking, boolean closingTransport) throws IOException {
if (closingTransport) {
protected void closeChannels(List<NioChannel> channels, boolean blocking, boolean doNotLinger) throws IOException {
if (doNotLinger) {
for (NioChannel channel : channels) {
/* We set SO_LINGER timeout to 0 to ensure that when we shutdown the node we don't have a gazillion connections sitting
* in TIME_WAIT to free up resources quickly. This is really the only part where we close the connection from the server
* side otherwise the client (node) initiates the TCP closing sequence which doesn't cause these issues. Setting this
* by default from the beginning can have unexpected side-effects an should be avoided, our protocol is designed
* in a way that clients close connection which is how it should be*/
if (channel.isOpen()) {
if (channel.isOpen() && channel.getRawChannel().supportedOptions().contains(StandardSocketOptions.SO_LINGER)) {
channel.getRawChannel().setOption(StandardSocketOptions.SO_LINGER, 0);
}
}
Expand Down

0 comments on commit 277637f

Please sign in to comment.