|
|
@@ -66,6 +66,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
|
|
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
|
|
import org.elasticsearch.common.util.concurrent.KeyedLock;
|
|
|
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
|
|
+import org.elasticsearch.common.util.iterable.Iterables;
|
|
|
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
|
|
import org.elasticsearch.monitor.jvm.JvmInfo;
|
|
|
import org.elasticsearch.rest.RestStatus;
|
|
|
@@ -85,7 +86,6 @@ import java.util.Collections;
|
|
|
import java.util.EnumMap;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
-import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Objects;
|
|
|
@@ -112,6 +112,7 @@ import static org.elasticsearch.common.settings.Setting.timeSetting;
|
|
|
import static org.elasticsearch.common.transport.NetworkExceptionHelper.isCloseConnectionException;
|
|
|
import static org.elasticsearch.common.transport.NetworkExceptionHelper.isConnectException;
|
|
|
import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
|
|
|
+import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet;
|
|
|
|
|
|
public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent implements Transport {
|
|
|
|
|
|
@@ -159,6 +160,8 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
protected volatile TransportServiceAdapter transportServiceAdapter;
|
|
|
// node id to actual channel
|
|
|
protected final ConcurrentMap<DiscoveryNode, NodeChannels> connectedNodes = newConcurrentMap();
|
|
|
+ private final Set<NodeChannels> openConnections = newConcurrentSet();
|
|
|
+
|
|
|
protected final Map<String, List<Channel>> serverChannels = newConcurrentMap();
|
|
|
protected final ConcurrentMap<String, BoundTransportAddress> profileBoundAddresses = newConcurrentMap();
|
|
|
|
|
|
@@ -357,9 +360,8 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
private final DiscoveryNode node;
|
|
|
private final AtomicBoolean closed = new AtomicBoolean(false);
|
|
|
private final Version version;
|
|
|
- private final Consumer<Connection> onClose;
|
|
|
|
|
|
- public NodeChannels(DiscoveryNode node, Channel[] channels, ConnectionProfile connectionProfile, Consumer<Connection> onClose) {
|
|
|
+ public NodeChannels(DiscoveryNode node, Channel[] channels, ConnectionProfile connectionProfile) {
|
|
|
this.node = node;
|
|
|
this.channels = channels;
|
|
|
assert channels.length == connectionProfile.getNumConnections() : "expected channels size to be == "
|
|
|
@@ -370,7 +372,6 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
typeMapping.put(type, handle);
|
|
|
}
|
|
|
version = node.getVersion();
|
|
|
- this.onClose = onClose;
|
|
|
}
|
|
|
|
|
|
NodeChannels(NodeChannels channels, Version handshakeVersion) {
|
|
|
@@ -378,7 +379,6 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
this.channels = channels.channels;
|
|
|
this.typeMapping = channels.typeMapping;
|
|
|
this.version = handshakeVersion;
|
|
|
- this.onClose = channels.onClose;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
@@ -413,7 +413,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
try {
|
|
|
closeChannels(Arrays.stream(channels).filter(Objects::nonNull).collect(Collectors.toList()));
|
|
|
} finally {
|
|
|
- onClose.accept(this);
|
|
|
+ onNodeChannelsClosed(this);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
@@ -455,27 +455,28 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
if (nodeChannels != null) {
|
|
|
return;
|
|
|
}
|
|
|
+ boolean success = false;
|
|
|
try {
|
|
|
- try {
|
|
|
- nodeChannels = openConnection(node, connectionProfile);
|
|
|
- connectionValidator.accept(nodeChannels, connectionProfile);
|
|
|
- } catch (Exception e) {
|
|
|
- logger.trace(
|
|
|
- (Supplier<?>) () -> new ParameterizedMessage(
|
|
|
- "failed to connect to [{}], cleaning dangling connections", node), e);
|
|
|
- IOUtils.closeWhileHandlingException(nodeChannels);
|
|
|
- throw e;
|
|
|
- }
|
|
|
+ nodeChannels = openConnection(node, connectionProfile);
|
|
|
+ connectionValidator.accept(nodeChannels, connectionProfile);
|
|
|
// we acquire a connection lock, so no way there is an existing connection
|
|
|
connectedNodes.put(node, nodeChannels);
|
|
|
if (logger.isDebugEnabled()) {
|
|
|
logger.debug("connected to node [{}]", node);
|
|
|
}
|
|
|
transportServiceAdapter.onNodeConnected(node);
|
|
|
+ success = true;
|
|
|
} catch (ConnectTransportException e) {
|
|
|
throw e;
|
|
|
} catch (Exception e) {
|
|
|
throw new ConnectTransportException(node, "general node connection failure", e);
|
|
|
+ } finally {
|
|
|
+ if (success == false) { // close the connection if there is a failure
|
|
|
+ logger.trace(
|
|
|
+ (Supplier<?>) () -> new ParameterizedMessage(
|
|
|
+ "failed to connect to [{}], cleaning dangling connections", node));
|
|
|
+ IOUtils.closeWhileHandlingException(nodeChannels);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
} finally {
|
|
|
@@ -518,7 +519,20 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
try {
|
|
|
ensureOpen();
|
|
|
try {
|
|
|
- nodeChannels = connectToChannels(node, connectionProfile);
|
|
|
+ AtomicBoolean runOnce = new AtomicBoolean(false);
|
|
|
+ Consumer<Channel> onClose = c -> {
|
|
|
+ assert isOpen(c) == false : "channel is still open when onClose is called";
|
|
|
+ try {
|
|
|
+ onChannelClosed(c);
|
|
|
+ } finally {
|
|
|
+ // we only need to disconnect from the nodes once since all other channels
|
|
|
+ // will also try to run this we protect it from running multiple times.
|
|
|
+ if (runOnce.compareAndSet(false, true)) {
|
|
|
+ disconnectFromNodeChannel(c, "channel closed");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ nodeChannels = connectToChannels(node, connectionProfile, onClose);
|
|
|
final Channel channel = nodeChannels.getChannels().get(0); // one channel is guaranteed by the connection profile
|
|
|
final TimeValue connectTimeout = connectionProfile.getConnectTimeout() == null ?
|
|
|
defaultConnectionProfile.getConnectTimeout() :
|
|
|
@@ -526,8 +540,9 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
final TimeValue handshakeTimeout = connectionProfile.getHandshakeTimeout() == null ?
|
|
|
connectTimeout : connectionProfile.getHandshakeTimeout();
|
|
|
final Version version = executeHandshake(node, channel, handshakeTimeout);
|
|
|
- transportServiceAdapter.onConnectionOpened(nodeChannels);
|
|
|
nodeChannels = new NodeChannels(nodeChannels, version); // clone the channels - we now have the correct version
|
|
|
+ transportServiceAdapter.onConnectionOpened(nodeChannels);
|
|
|
+ openConnections.add(nodeChannels);
|
|
|
success = true;
|
|
|
return nodeChannels;
|
|
|
} catch (ConnectTransportException e) {
|
|
|
@@ -580,24 +595,37 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
/**
|
|
|
* Disconnects from a node if a channel is found as part of that nodes channels.
|
|
|
*/
|
|
|
- protected final void disconnectFromNodeChannel(final Channel channel, final Exception failure) {
|
|
|
+ protected final void disconnectFromNodeChannel(final Channel channel, final String reason) {
|
|
|
threadPool.generic().execute(() -> {
|
|
|
try {
|
|
|
- try {
|
|
|
+ if (isOpen(channel)) {
|
|
|
closeChannels(Collections.singletonList(channel));
|
|
|
- } finally {
|
|
|
- for (DiscoveryNode node : connectedNodes.keySet()) {
|
|
|
- if (disconnectFromNode(node, channel, ExceptionsHelper.detailedMessage(failure))) {
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ logger.warn("failed to close channel", e);
|
|
|
+ } finally {
|
|
|
+ outer:
|
|
|
+ {
|
|
|
+ for (Map.Entry<DiscoveryNode, NodeChannels> entry : connectedNodes.entrySet()) {
|
|
|
+ if (disconnectFromNode(entry.getKey(), channel, reason)) {
|
|
|
// if we managed to find this channel and disconnect from it, then break, no need to check on
|
|
|
// the rest of the nodes
|
|
|
+ // #onNodeChannelsClosed will remove it..
|
|
|
+ assert openConnections.contains(entry.getValue()) == false : "NodeChannel#close should remove the connetion";
|
|
|
+ // we can only be connected and published to a single node with one connection. So if disconnectFromNode
|
|
|
+ // returns true we can safely break out from here since we cleaned up everything needed
|
|
|
+ break outer;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // now if we haven't found the right connection in the connected nodes we have to go through the open connections
|
|
|
+ // it might be that the channel belongs to a connection that is not published
|
|
|
+ for (NodeChannels channels : openConnections) {
|
|
|
+ if (channels.hasChannel(channel)) {
|
|
|
+ IOUtils.closeWhileHandlingException(channels);
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- } catch (IOException e) {
|
|
|
- logger.warn("failed to close channel", e);
|
|
|
- } finally {
|
|
|
- onChannelClosed(channel);
|
|
|
}
|
|
|
});
|
|
|
}
|
|
|
@@ -901,12 +929,11 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
"Error closing serverChannel for profile [{}]", entry.getKey()), e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- for (Iterator<NodeChannels> it = connectedNodes.values().iterator(); it.hasNext();) {
|
|
|
- NodeChannels nodeChannels = it.next();
|
|
|
- it.remove();
|
|
|
- IOUtils.closeWhileHandlingException(nodeChannels);
|
|
|
- }
|
|
|
+ // we are holding a write lock so nobody modifies the connectedNodes / openConnections map - it's safe to first close
|
|
|
+ // all instances and then clear them maps
|
|
|
+ IOUtils.closeWhileHandlingException(Iterables.concat(connectedNodes.values(), openConnections));
|
|
|
+ openConnections.clear();
|
|
|
+ connectedNodes.clear();
|
|
|
stopInternal();
|
|
|
} finally {
|
|
|
globalLock.writeLock().unlock();
|
|
|
@@ -923,11 +950,13 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
}
|
|
|
|
|
|
protected void onException(Channel channel, Exception e) {
|
|
|
+ String reason = ExceptionsHelper.detailedMessage(e);
|
|
|
if (!lifecycle.started()) {
|
|
|
// just close and ignore - we are already stopped and just need to make sure we release all resources
|
|
|
- disconnectFromNodeChannel(channel, e);
|
|
|
+ disconnectFromNodeChannel(channel, reason);
|
|
|
return;
|
|
|
}
|
|
|
+
|
|
|
if (isCloseConnectionException(e)) {
|
|
|
logger.trace(
|
|
|
(Supplier<?>) () -> new ParameterizedMessage(
|
|
|
@@ -935,15 +964,15 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
channel),
|
|
|
e);
|
|
|
// close the channel, which will cause a node to be disconnected if relevant
|
|
|
- disconnectFromNodeChannel(channel, e);
|
|
|
+ disconnectFromNodeChannel(channel, reason);
|
|
|
} else if (isConnectException(e)) {
|
|
|
logger.trace((Supplier<?>) () -> new ParameterizedMessage("connect exception caught on transport layer [{}]", channel), e);
|
|
|
// close the channel as safe measure, which will cause a node to be disconnected if relevant
|
|
|
- disconnectFromNodeChannel(channel, e);
|
|
|
+ disconnectFromNodeChannel(channel, reason);
|
|
|
} else if (e instanceof BindException) {
|
|
|
logger.trace((Supplier<?>) () -> new ParameterizedMessage("bind exception caught on transport layer [{}]", channel), e);
|
|
|
// close the channel as safe measure, which will cause a node to be disconnected if relevant
|
|
|
- disconnectFromNodeChannel(channel, e);
|
|
|
+ disconnectFromNodeChannel(channel, reason);
|
|
|
} else if (e instanceof CancelledKeyException) {
|
|
|
logger.trace(
|
|
|
(Supplier<?>) () -> new ParameterizedMessage(
|
|
|
@@ -951,7 +980,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
channel),
|
|
|
e);
|
|
|
// close the channel as safe measure, which will cause a node to be disconnected if relevant
|
|
|
- disconnectFromNodeChannel(channel, e);
|
|
|
+ disconnectFromNodeChannel(channel, reason);
|
|
|
} else if (e instanceof TcpTransport.HttpOnTransportException) {
|
|
|
// in case we are able to return data, serialize the exception content and sent it back to the client
|
|
|
if (isOpen(channel)) {
|
|
|
@@ -981,7 +1010,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
logger.warn(
|
|
|
(Supplier<?>) () -> new ParameterizedMessage("exception caught on transport layer [{}], closing connection", channel), e);
|
|
|
// close the channel, which will cause a node to be disconnected if relevant
|
|
|
- disconnectFromNodeChannel(channel, e);
|
|
|
+ disconnectFromNodeChannel(channel, reason);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -1012,7 +1041,8 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
*/
|
|
|
protected abstract void sendMessage(Channel channel, BytesReference reference, ActionListener<Channel> listener);
|
|
|
|
|
|
- protected abstract NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile connectionProfile) throws IOException;
|
|
|
+ protected abstract NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile connectionProfile,
|
|
|
+ Consumer<Channel> onChannelClose) throws IOException;
|
|
|
|
|
|
/**
|
|
|
* Called to tear down internal resources
|
|
|
@@ -1607,7 +1637,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
/**
|
|
|
* Called once the channel is closed for instance due to a disconnect or a closed socket etc.
|
|
|
*/
|
|
|
- protected final void onChannelClosed(Channel channel) {
|
|
|
+ private void onChannelClosed(Channel channel) {
|
|
|
final Optional<Long> first = pendingHandshakes.entrySet().stream()
|
|
|
.filter((entry) -> entry.getValue().channel == channel).map((e) -> e.getKey()).findFirst();
|
|
|
if (first.isPresent()) {
|
|
|
@@ -1655,4 +1685,20 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
|
|
|
Releasables.close(optionalReleasable, transportAdaptorCallback::run);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private void onNodeChannelsClosed(NodeChannels channels) {
|
|
|
+ // don't assert here since the channel / connection might not have been registered yet
|
|
|
+ final boolean remove = openConnections.remove(channels);
|
|
|
+ if (remove) {
|
|
|
+ transportServiceAdapter.onConnectionClosed(channels);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ final int getNumOpenConnections() {
|
|
|
+ return openConnections.size();
|
|
|
+ }
|
|
|
+
|
|
|
+ final int getNumConnectedNodes() {
|
|
|
+ return connectedNodes.size();
|
|
|
+ }
|
|
|
}
|