mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-25 01:19:02 +00:00
Do not set SO_LINGER on server channels (#26997)
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.
This commit is contained in:
parent
bb013c60b5
commit
277637f42f
@ -905,11 +905,9 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
||||
// 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
|
||||
@ -1024,9 +1022,9 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
||||
*
|
||||
* @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
|
||||
|
@ -191,7 +191,7 @@ public class TcpTransportTests extends ESTestCase {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void closeChannels(List channel, boolean blocking, boolean closingTransport) throws IOException {
|
||||
protected void closeChannels(List channel, boolean blocking, boolean doNotLinger) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
|
@ -331,8 +331,8 @@ public class Netty4Transport extends TcpTransport<Channel> {
|
||||
}
|
||||
|
||||
@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
|
||||
|
@ -243,8 +243,8 @@ public class MockTcpTransport extends TcpTransport<MockTcpTransport.MockChannel>
|
||||
}
|
||||
|
||||
@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
|
||||
|
@ -99,15 +99,15 @@ public class NioTransport extends TcpTransport<NioChannel> {
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user