mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-02-16 09:54:55 +00:00
Asynchronously connect to remote clusters (#44825)
Refactors RemoteClusterConnection so that it no longer blockingly connects to remote clusters. Relates to #40150
This commit is contained in:
parent
0ce841915c
commit
bd8470e738
@ -25,11 +25,11 @@ import org.apache.lucene.store.AlreadyClosedException;
|
|||||||
import org.apache.lucene.util.SetOnce;
|
import org.apache.lucene.util.SetOnce;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.StepListener;
|
||||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
|
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
|
||||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
|
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
|
||||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||||
import org.elasticsearch.action.support.ContextPreservingActionListener;
|
import org.elasticsearch.action.support.ContextPreservingActionListener;
|
||||||
import org.elasticsearch.action.support.PlainActionFuture;
|
|
||||||
import org.elasticsearch.cluster.ClusterName;
|
import org.elasticsearch.cluster.ClusterName;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
@ -38,7 +38,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.transport.TransportAddress;
|
import org.elasticsearch.common.transport.TransportAddress;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.util.CancellableThreads;
|
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||||
import org.elasticsearch.core.internal.io.IOUtils;
|
import org.elasticsearch.core.internal.io.IOUtils;
|
||||||
@ -48,17 +47,14 @@ import java.io.Closeable;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.ArrayBlockingQueue;
|
|
||||||
import java.util.concurrent.BlockingQueue;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.RejectedExecutionException;
|
import java.util.concurrent.RejectedExecutionException;
|
||||||
import java.util.concurrent.Semaphore;
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
@ -138,7 +134,7 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
|
|||||||
if (proxyAddress == null || proxyAddress.isEmpty()) {
|
if (proxyAddress == null || proxyAddress.isEmpty()) {
|
||||||
return node;
|
return node;
|
||||||
} else {
|
} else {
|
||||||
// resovle proxy address lazy here
|
// resolve proxy address lazy here
|
||||||
InetSocketAddress proxyInetAddress = RemoteClusterAware.parseSeedAddress(proxyAddress);
|
InetSocketAddress proxyInetAddress = RemoteClusterAware.parseSeedAddress(proxyAddress);
|
||||||
return new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getHostName(), node
|
return new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getHostName(), node
|
||||||
.getHostAddress(), new TransportAddress(proxyInetAddress), node.getAttributes(), node.getRoles(), node.getVersion());
|
.getHostAddress(), new TransportAddress(proxyInetAddress), node.getAttributes(), node.getRoles(), node.getVersion());
|
||||||
@ -175,7 +171,9 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
|
|||||||
public void onNodeDisconnected(DiscoveryNode node) {
|
public void onNodeDisconnected(DiscoveryNode node) {
|
||||||
if (connectionManager.size() < maxNumRemoteConnections) {
|
if (connectionManager.size() < maxNumRemoteConnections) {
|
||||||
// try to reconnect and fill up the slot of the disconnected node
|
// try to reconnect and fill up the slot of the disconnected node
|
||||||
connectHandler.forceConnect();
|
connectHandler.connect(ActionListener.wrap(
|
||||||
|
ignore -> logger.trace("successfully connected after disconnect of {}", node),
|
||||||
|
e -> logger.trace(() -> new ParameterizedMessage("failed to connect after disconnect of {}", node), e)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -344,201 +342,178 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
|
|||||||
* we will just reject the connect trigger which will lead to failing searches.
|
* we will just reject the connect trigger which will lead to failing searches.
|
||||||
*/
|
*/
|
||||||
private class ConnectHandler implements Closeable {
|
private class ConnectHandler implements Closeable {
|
||||||
private final Semaphore running = new Semaphore(1);
|
private static final int MAX_LISTENERS = 100;
|
||||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||||
private final BlockingQueue<ActionListener<Void>> queue = new ArrayBlockingQueue<>(100);
|
private final Object mutex = new Object();
|
||||||
private final CancellableThreads cancellableThreads = new CancellableThreads();
|
private List<ActionListener<Void>> listeners = new ArrayList<>();
|
||||||
|
|
||||||
/**
|
|
||||||
* Triggers a connect round iff there are pending requests queued up and if there is no
|
|
||||||
* connect round currently running.
|
|
||||||
*/
|
|
||||||
void maybeConnect() {
|
|
||||||
connect(null);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Triggers a connect round unless there is one running already. If there is a connect round running, the listener will either
|
* Triggers a connect round unless there is one running already. If there is a connect round running, the listener will either
|
||||||
* be queued or rejected and failed.
|
* be queued or rejected and failed.
|
||||||
*/
|
*/
|
||||||
void connect(ActionListener<Void> connectListener) {
|
void connect(ActionListener<Void> connectListener) {
|
||||||
connect(connectListener, false);
|
boolean runConnect = false;
|
||||||
}
|
final ActionListener<Void> listener =
|
||||||
|
|
||||||
/**
|
|
||||||
* Triggers a connect round unless there is one already running. In contrast to {@link #maybeConnect()} will this method also
|
|
||||||
* trigger a connect round if there is no listener queued up.
|
|
||||||
*/
|
|
||||||
void forceConnect() {
|
|
||||||
connect(null, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void connect(ActionListener<Void> connectListener, boolean forceRun) {
|
|
||||||
final boolean runConnect;
|
|
||||||
final Collection<ActionListener<Void>> toNotify;
|
|
||||||
final ActionListener<Void> listener = connectListener == null ? null :
|
|
||||||
ContextPreservingActionListener.wrapPreservingContext(connectListener, threadPool.getThreadContext());
|
ContextPreservingActionListener.wrapPreservingContext(connectListener, threadPool.getThreadContext());
|
||||||
synchronized (queue) {
|
synchronized (mutex) {
|
||||||
if (listener != null && queue.offer(listener) == false) {
|
if (closed.get()) {
|
||||||
listener.onFailure(new RejectedExecutionException("connect queue is full"));
|
assert listeners.isEmpty();
|
||||||
return;
|
|
||||||
}
|
|
||||||
if (forceRun == false && queue.isEmpty()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
runConnect = running.tryAcquire();
|
|
||||||
if (runConnect) {
|
|
||||||
toNotify = new ArrayList<>();
|
|
||||||
queue.drainTo(toNotify);
|
|
||||||
if (closed.get()) {
|
|
||||||
running.release();
|
|
||||||
ActionListener.onFailure(toNotify, new AlreadyClosedException("connect handler is already closed"));
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
toNotify = Collections.emptyList();
|
if (listeners.size() >= MAX_LISTENERS) {
|
||||||
|
assert listeners.size() == MAX_LISTENERS;
|
||||||
|
listener.onFailure(new RejectedExecutionException("connect queue is full"));
|
||||||
|
return;
|
||||||
|
} else {
|
||||||
|
listeners.add(listener);
|
||||||
|
}
|
||||||
|
runConnect = listeners.size() == 1;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if (closed.get()) {
|
||||||
|
connectListener.onFailure(new AlreadyClosedException("connect handler is already closed"));
|
||||||
|
return;
|
||||||
|
}
|
||||||
if (runConnect) {
|
if (runConnect) {
|
||||||
forkConnect(toNotify);
|
ExecutorService executor = threadPool.executor(ThreadPool.Names.MANAGEMENT);
|
||||||
|
executor.submit(new AbstractRunnable() {
|
||||||
|
@Override
|
||||||
|
public void onFailure(Exception e) {
|
||||||
|
ActionListener.onFailure(getAndClearListeners(), e);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doRun() {
|
||||||
|
collectRemoteNodes(seedNodes.stream().map(Tuple::v2).iterator(),
|
||||||
|
new ActionListener<Void>() {
|
||||||
|
@Override
|
||||||
|
public void onResponse(Void aVoid) {
|
||||||
|
ActionListener.onResponse(getAndClearListeners(), aVoid);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(Exception e) {
|
||||||
|
ActionListener.onFailure(getAndClearListeners(), e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void forkConnect(final Collection<ActionListener<Void>> toNotify) {
|
private List<ActionListener<Void>> getAndClearListeners() {
|
||||||
ExecutorService executor = threadPool.executor(ThreadPool.Names.MANAGEMENT);
|
final List<ActionListener<Void>> result;
|
||||||
executor.submit(new AbstractRunnable() {
|
synchronized (mutex) {
|
||||||
@Override
|
if (listeners.isEmpty()) {
|
||||||
public void onFailure(Exception e) {
|
result = Collections.emptyList();
|
||||||
synchronized (queue) {
|
} else {
|
||||||
running.release();
|
result = listeners;
|
||||||
}
|
listeners = new ArrayList<>();
|
||||||
try {
|
|
||||||
ActionListener.onFailure(toNotify, e);
|
|
||||||
} finally {
|
|
||||||
maybeConnect();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
@Override
|
return result;
|
||||||
protected void doRun() {
|
|
||||||
ActionListener<Void> listener = ActionListener.wrap((x) -> {
|
|
||||||
synchronized (queue) {
|
|
||||||
running.release();
|
|
||||||
}
|
|
||||||
try {
|
|
||||||
ActionListener.onResponse(toNotify, x);
|
|
||||||
} finally {
|
|
||||||
maybeConnect();
|
|
||||||
}
|
|
||||||
|
|
||||||
}, (e) -> {
|
|
||||||
synchronized (queue) {
|
|
||||||
running.release();
|
|
||||||
}
|
|
||||||
try {
|
|
||||||
ActionListener.onFailure(toNotify, e);
|
|
||||||
} finally {
|
|
||||||
maybeConnect();
|
|
||||||
}
|
|
||||||
});
|
|
||||||
collectRemoteNodes(seedNodes.stream().map(Tuple::v2).iterator(), transportService, connectionManager, listener);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void collectRemoteNodes(Iterator<Supplier<DiscoveryNode>> seedNodes, final TransportService transportService,
|
private void collectRemoteNodes(Iterator<Supplier<DiscoveryNode>> seedNodes, ActionListener<Void> listener) {
|
||||||
final ConnectionManager manager, ActionListener<Void> listener) {
|
|
||||||
if (Thread.currentThread().isInterrupted()) {
|
if (Thread.currentThread().isInterrupted()) {
|
||||||
listener.onFailure(new InterruptedException("remote connect thread got interrupted"));
|
listener.onFailure(new InterruptedException("remote connect thread got interrupted"));
|
||||||
}
|
}
|
||||||
try {
|
|
||||||
if (seedNodes.hasNext()) {
|
|
||||||
cancellableThreads.executeIO(() -> {
|
|
||||||
final DiscoveryNode seedNode = maybeAddProxyAddress(proxyAddress, seedNodes.next().get());
|
|
||||||
logger.debug("[{}] opening connection to seed node: [{}] proxy address: [{}]", clusterAlias, seedNode,
|
|
||||||
proxyAddress);
|
|
||||||
final TransportService.HandshakeResponse handshakeResponse;
|
|
||||||
final ConnectionProfile profile = ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG);
|
|
||||||
final Transport.Connection connection = PlainActionFuture.get(
|
|
||||||
fut -> manager.openConnection(seedNode, profile, fut));
|
|
||||||
boolean success = false;
|
|
||||||
try {
|
|
||||||
try {
|
|
||||||
ConnectionProfile connectionProfile = connectionManager.getConnectionProfile();
|
|
||||||
handshakeResponse = PlainActionFuture.get(fut ->
|
|
||||||
transportService.handshake(connection, connectionProfile.getHandshakeTimeout().millis(),
|
|
||||||
getRemoteClusterNamePredicate(), fut));
|
|
||||||
} catch (IllegalStateException ex) {
|
|
||||||
logger.warn(new ParameterizedMessage("failed to connect to seed node [{}]", connection.getNode()), ex);
|
|
||||||
throw ex;
|
|
||||||
}
|
|
||||||
|
|
||||||
final DiscoveryNode handshakeNode = maybeAddProxyAddress(proxyAddress, handshakeResponse.getDiscoveryNode());
|
if (seedNodes.hasNext()) {
|
||||||
if (nodePredicate.test(handshakeNode) && manager.size() < maxNumRemoteConnections) {
|
final DiscoveryNode seedNode = maybeAddProxyAddress(proxyAddress, seedNodes.next().get());
|
||||||
PlainActionFuture.get(fut -> manager.connectToNode(handshakeNode, null,
|
logger.debug("[{}] opening connection to seed node: [{}] proxy address: [{}]", clusterAlias, seedNode,
|
||||||
transportService.connectionValidator(handshakeNode), ActionListener.map(fut, x -> null)));
|
proxyAddress);
|
||||||
if (remoteClusterName.get() == null) {
|
final ConnectionProfile profile = ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG);
|
||||||
assert handshakeResponse.getClusterName().value() != null;
|
|
||||||
remoteClusterName.set(handshakeResponse.getClusterName());
|
final StepListener<Transport.Connection> openConnectionStep = new StepListener<>();
|
||||||
}
|
connectionManager.openConnection(seedNode, profile, openConnectionStep);
|
||||||
}
|
|
||||||
ClusterStateRequest request = new ClusterStateRequest();
|
final Consumer<Exception> onFailure = e -> {
|
||||||
request.clear();
|
if (e instanceof ConnectTransportException ||
|
||||||
request.nodes(true);
|
e instanceof IOException ||
|
||||||
// here we pass on the connection since we can only close it once the sendRequest returns otherwise
|
e instanceof IllegalStateException) {
|
||||||
// due to the async nature (it will return before it's actually sent) this can cause the request to fail
|
// ISE if we fail the handshake with an version incompatible node
|
||||||
// due to an already closed connection.
|
if (seedNodes.hasNext()) {
|
||||||
ThreadPool threadPool = transportService.getThreadPool();
|
logger.debug(() -> new ParameterizedMessage(
|
||||||
ThreadContext threadContext = threadPool.getThreadContext();
|
"fetching nodes from external cluster [{}] failed moving to next node", clusterAlias), e);
|
||||||
TransportService.ContextRestoreResponseHandler<ClusterStateResponse> responseHandler = new TransportService
|
collectRemoteNodes(seedNodes, listener);
|
||||||
.ContextRestoreResponseHandler<>(threadContext.newRestorableContext(false),
|
return;
|
||||||
new SniffClusterStateResponseHandler(connection, listener, seedNodes,
|
|
||||||
cancellableThreads));
|
|
||||||
try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
|
|
||||||
// we stash any context here since this is an internal execution and should not leak any
|
|
||||||
// existing context information.
|
|
||||||
threadContext.markAsSystemContext();
|
|
||||||
transportService.sendRequest(connection, ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY,
|
|
||||||
responseHandler);
|
|
||||||
}
|
|
||||||
success = true;
|
|
||||||
} finally {
|
|
||||||
if (success == false) {
|
|
||||||
connection.close();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
});
|
}
|
||||||
} else {
|
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster [{}] failed", clusterAlias), e);
|
||||||
listener.onFailure(new IllegalStateException("no seed node left"));
|
listener.onFailure(e);
|
||||||
}
|
};
|
||||||
} catch (CancellableThreads.ExecutionCancelledException ex) {
|
|
||||||
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster [{}] failed", clusterAlias), ex);
|
final StepListener<TransportService.HandshakeResponse> handShakeStep = new StepListener<>();
|
||||||
listener.onFailure(ex); // we got canceled - fail the listener and step out
|
openConnectionStep.whenComplete(connection -> {
|
||||||
} catch (ConnectTransportException | IOException | IllegalStateException ex) {
|
ConnectionProfile connectionProfile = connectionManager.getConnectionProfile();
|
||||||
// ISE if we fail the handshake with an version incompatible node
|
transportService.handshake(connection, connectionProfile.getHandshakeTimeout().millis(),
|
||||||
if (seedNodes.hasNext()) {
|
getRemoteClusterNamePredicate(), handShakeStep);
|
||||||
logger.debug(() -> new ParameterizedMessage("fetching nodes from external cluster [{}] failed moving to next node",
|
}, onFailure);
|
||||||
clusterAlias), ex);
|
|
||||||
collectRemoteNodes(seedNodes, transportService, manager, listener);
|
final StepListener<Void> fullConnectionStep = new StepListener<>();
|
||||||
} else {
|
handShakeStep.whenComplete(handshakeResponse -> {
|
||||||
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster [{}] failed", clusterAlias), ex);
|
final DiscoveryNode handshakeNode = maybeAddProxyAddress(proxyAddress, handshakeResponse.getDiscoveryNode());
|
||||||
listener.onFailure(ex);
|
|
||||||
}
|
if (nodePredicate.test(handshakeNode) && connectionManager.size() < maxNumRemoteConnections) {
|
||||||
|
connectionManager.connectToNode(handshakeNode, null,
|
||||||
|
transportService.connectionValidator(handshakeNode), fullConnectionStep);
|
||||||
|
} else {
|
||||||
|
fullConnectionStep.onResponse(null);
|
||||||
|
}
|
||||||
|
}, e -> {
|
||||||
|
final Transport.Connection connection = openConnectionStep.result();
|
||||||
|
logger.warn(new ParameterizedMessage("failed to connect to seed node [{}]", connection.getNode()), e);
|
||||||
|
IOUtils.closeWhileHandlingException(connection);
|
||||||
|
onFailure.accept(e);
|
||||||
|
});
|
||||||
|
|
||||||
|
fullConnectionStep.whenComplete(aVoid -> {
|
||||||
|
if (remoteClusterName.get() == null) {
|
||||||
|
TransportService.HandshakeResponse handshakeResponse = handShakeStep.result();
|
||||||
|
assert handshakeResponse.getClusterName().value() != null;
|
||||||
|
remoteClusterName.set(handshakeResponse.getClusterName());
|
||||||
|
}
|
||||||
|
final Transport.Connection connection = openConnectionStep.result();
|
||||||
|
|
||||||
|
ClusterStateRequest request = new ClusterStateRequest();
|
||||||
|
request.clear();
|
||||||
|
request.nodes(true);
|
||||||
|
// here we pass on the connection since we can only close it once the sendRequest returns otherwise
|
||||||
|
// due to the async nature (it will return before it's actually sent) this can cause the request to fail
|
||||||
|
// due to an already closed connection.
|
||||||
|
ThreadPool threadPool = transportService.getThreadPool();
|
||||||
|
ThreadContext threadContext = threadPool.getThreadContext();
|
||||||
|
TransportService.ContextRestoreResponseHandler<ClusterStateResponse> responseHandler = new TransportService
|
||||||
|
.ContextRestoreResponseHandler<>(threadContext.newRestorableContext(false),
|
||||||
|
new SniffClusterStateResponseHandler(connection, listener, seedNodes));
|
||||||
|
try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
|
||||||
|
// we stash any context here since this is an internal execution and should not leak any
|
||||||
|
// existing context information.
|
||||||
|
threadContext.markAsSystemContext();
|
||||||
|
transportService.sendRequest(connection, ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY,
|
||||||
|
responseHandler);
|
||||||
|
}
|
||||||
|
}, e -> {
|
||||||
|
IOUtils.closeWhileHandlingException(openConnectionStep.result());
|
||||||
|
onFailure.accept(e);
|
||||||
|
});
|
||||||
|
} else {
|
||||||
|
listener.onFailure(new IllegalStateException("no seed node left"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
try {
|
final List<ActionListener<Void>> toNotify;
|
||||||
|
synchronized (mutex) {
|
||||||
if (closed.compareAndSet(false, true)) {
|
if (closed.compareAndSet(false, true)) {
|
||||||
cancellableThreads.cancel("connect handler is closed");
|
toNotify = listeners;
|
||||||
running.acquire(); // acquire the semaphore to ensure all connections are closed and all thread joined
|
listeners = Collections.emptyList();
|
||||||
running.release();
|
} else {
|
||||||
maybeConnect(); // now go and notify pending listeners
|
toNotify = Collections.emptyList();
|
||||||
}
|
}
|
||||||
} catch (InterruptedException e) {
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
}
|
}
|
||||||
|
ActionListener.onFailure(toNotify, new AlreadyClosedException("connect handler is already closed"));
|
||||||
}
|
}
|
||||||
|
|
||||||
final boolean isClosed() {
|
final boolean isClosed() {
|
||||||
@ -551,15 +526,12 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
|
|||||||
private final Transport.Connection connection;
|
private final Transport.Connection connection;
|
||||||
private final ActionListener<Void> listener;
|
private final ActionListener<Void> listener;
|
||||||
private final Iterator<Supplier<DiscoveryNode>> seedNodes;
|
private final Iterator<Supplier<DiscoveryNode>> seedNodes;
|
||||||
private final CancellableThreads cancellableThreads;
|
|
||||||
|
|
||||||
SniffClusterStateResponseHandler(Transport.Connection connection, ActionListener<Void> listener,
|
SniffClusterStateResponseHandler(Transport.Connection connection, ActionListener<Void> listener,
|
||||||
Iterator<Supplier<DiscoveryNode>> seedNodes,
|
Iterator<Supplier<DiscoveryNode>> seedNodes) {
|
||||||
CancellableThreads cancellableThreads) {
|
|
||||||
this.connection = connection;
|
this.connection = connection;
|
||||||
this.listener = listener;
|
this.listener = listener;
|
||||||
this.seedNodes = seedNodes;
|
this.seedNodes = seedNodes;
|
||||||
this.cancellableThreads = cancellableThreads;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -569,43 +541,44 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void handleResponse(ClusterStateResponse response) {
|
public void handleResponse(ClusterStateResponse response) {
|
||||||
try {
|
handleNodes(response.getState().nodes().getNodes().valuesIt());
|
||||||
if (remoteClusterName.get() == null) {
|
}
|
||||||
assert response.getClusterName().value() != null;
|
|
||||||
remoteClusterName.set(response.getClusterName());
|
private void handleNodes(Iterator<DiscoveryNode> nodesIter) {
|
||||||
}
|
while (nodesIter.hasNext()) {
|
||||||
try (Closeable theConnection = connection) { // the connection is unused - see comment in #collectRemoteNodes
|
final DiscoveryNode node = maybeAddProxyAddress(proxyAddress, nodesIter.next());
|
||||||
// we have to close this connection before we notify listeners - this is mainly needed for test correctness
|
if (nodePredicate.test(node) && connectionManager.size() < maxNumRemoteConnections) {
|
||||||
// since if we do it afterwards we might fail assertions that check if all high level connections are closed.
|
connectionManager.connectToNode(node, null,
|
||||||
// from a code correctness perspective we could also close it afterwards. This try/with block will
|
transportService.connectionValidator(node), new ActionListener<Void>() {
|
||||||
// maintain the possibly exceptions thrown from within the try block and suppress the ones that are possible thrown
|
@Override
|
||||||
// by closing the connection
|
public void onResponse(Void aVoid) {
|
||||||
cancellableThreads.executeIO(() -> {
|
handleNodes(nodesIter);
|
||||||
DiscoveryNodes nodes = response.getState().nodes();
|
}
|
||||||
Iterable<DiscoveryNode> nodesIter = nodes.getNodes()::valuesIt;
|
|
||||||
for (DiscoveryNode n : nodesIter) {
|
@Override
|
||||||
DiscoveryNode node = maybeAddProxyAddress(proxyAddress, n);
|
public void onFailure(Exception e) {
|
||||||
if (nodePredicate.test(node) && connectionManager.size() < maxNumRemoteConnections) {
|
if (e instanceof ConnectTransportException ||
|
||||||
try {
|
e instanceof IllegalStateException) {
|
||||||
// noop if node is connected
|
|
||||||
PlainActionFuture.get(fut -> connectionManager.connectToNode(node, null,
|
|
||||||
transportService.connectionValidator(node), ActionListener.map(fut, x -> null)));
|
|
||||||
} catch (ConnectTransportException | IllegalStateException ex) {
|
|
||||||
// ISE if we fail the handshake with an version incompatible node
|
// ISE if we fail the handshake with an version incompatible node
|
||||||
// fair enough we can't connect just move on
|
// fair enough we can't connect just move on
|
||||||
logger.debug(() -> new ParameterizedMessage("failed to connect to node {}", node), ex);
|
logger.debug(() -> new ParameterizedMessage("failed to connect to node {}", node), e);
|
||||||
|
handleNodes(nodesIter);
|
||||||
|
} else {
|
||||||
|
logger.warn(() ->
|
||||||
|
new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterAlias), e);
|
||||||
|
IOUtils.closeWhileHandlingException(connection);
|
||||||
|
collectRemoteNodes(seedNodes, listener);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
});
|
||||||
});
|
return;
|
||||||
}
|
}
|
||||||
listener.onResponse(null);
|
|
||||||
} catch (CancellableThreads.ExecutionCancelledException ex) {
|
|
||||||
listener.onFailure(ex); // we got canceled - fail the listener and step out
|
|
||||||
} catch (Exception ex) {
|
|
||||||
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterAlias), ex);
|
|
||||||
collectRemoteNodes(seedNodes, transportService, connectionManager, listener);
|
|
||||||
}
|
}
|
||||||
|
// We have to close this connection before we notify listeners - this is mainly needed for test correctness
|
||||||
|
// since if we do it afterwards we might fail assertions that check if all high level connections are closed.
|
||||||
|
// from a code correctness perspective we could also close it afterwards.
|
||||||
|
IOUtils.closeWhileHandlingException(connection);
|
||||||
|
listener.onResponse(null);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -615,7 +588,7 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
|
|||||||
IOUtils.closeWhileHandlingException(connection);
|
IOUtils.closeWhileHandlingException(connection);
|
||||||
} finally {
|
} finally {
|
||||||
// once the connection is closed lets try the next node
|
// once the connection is closed lets try the next node
|
||||||
collectRemoteNodes(seedNodes, transportService, connectionManager, listener);
|
collectRemoteNodes(seedNodes, listener);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -627,7 +600,9 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
|
|||||||
}
|
}
|
||||||
|
|
||||||
boolean assertNoRunningConnections() { // for testing only
|
boolean assertNoRunningConnections() { // for testing only
|
||||||
assert connectHandler.running.availablePermits() == 1;
|
synchronized (connectHandler.mutex) {
|
||||||
|
assert connectHandler.listeners.isEmpty();
|
||||||
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -47,7 +47,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.transport.TransportAddress;
|
import org.elasticsearch.common.transport.TransportAddress;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.util.CancellableThreads;
|
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
import org.elasticsearch.core.internal.io.IOUtils;
|
import org.elasticsearch.core.internal.io.IOUtils;
|
||||||
@ -432,20 +431,6 @@ public class RemoteClusterConnectionTests extends ESTestCase {
|
|||||||
ActionListener<Void> listener = ActionListener.wrap(
|
ActionListener<Void> listener = ActionListener.wrap(
|
||||||
x -> latch.countDown(),
|
x -> latch.countDown(),
|
||||||
x -> {
|
x -> {
|
||||||
/*
|
|
||||||
* This can occur on a thread submitted to the thread pool while we are closing the
|
|
||||||
* remote cluster connection at the end of the test.
|
|
||||||
*/
|
|
||||||
if (x instanceof CancellableThreads.ExecutionCancelledException) {
|
|
||||||
try {
|
|
||||||
// we should already be shutting down
|
|
||||||
assertEquals(0L, latch.getCount());
|
|
||||||
} finally {
|
|
||||||
// ensure we count down the latch on failure as well to not prevent failing tests from ending
|
|
||||||
latch.countDown();
|
|
||||||
}
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
exceptionAtomicReference.set(x);
|
exceptionAtomicReference.set(x);
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
@ -580,7 +565,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
|
|||||||
closeRemote.countDown();
|
closeRemote.countDown();
|
||||||
listenerCalled.await();
|
listenerCalled.await();
|
||||||
assertNotNull(exceptionReference.get());
|
assertNotNull(exceptionReference.get());
|
||||||
expectThrows(CancellableThreads.ExecutionCancelledException.class, () -> {
|
expectThrows(AlreadyClosedException.class, () -> {
|
||||||
throw exceptionReference.get();
|
throw exceptionReference.get();
|
||||||
});
|
});
|
||||||
|
|
||||||
@ -640,16 +625,6 @@ public class RemoteClusterConnectionTests extends ESTestCase {
|
|||||||
latch.countDown();
|
latch.countDown();
|
||||||
},
|
},
|
||||||
x -> {
|
x -> {
|
||||||
/*
|
|
||||||
* This can occur on a thread submitted to the thread pool while we are closing the
|
|
||||||
* remote cluster connection at the end of the test.
|
|
||||||
*/
|
|
||||||
if (x instanceof CancellableThreads.ExecutionCancelledException) {
|
|
||||||
// we should already be shutting down
|
|
||||||
assertTrue(executed.get());
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
assertTrue(executed.compareAndSet(false, true));
|
assertTrue(executed.compareAndSet(false, true));
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
|
|
||||||
@ -737,8 +712,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
|
|||||||
throw assertionError;
|
throw assertionError;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (x instanceof RejectedExecutionException || x instanceof AlreadyClosedException
|
if (x instanceof RejectedExecutionException || x instanceof AlreadyClosedException) {
|
||||||
|| x instanceof CancellableThreads.ExecutionCancelledException) {
|
|
||||||
// that's fine
|
// that's fine
|
||||||
} else {
|
} else {
|
||||||
throw new AssertionError(x);
|
throw new AssertionError(x);
|
||||||
|
Loading…
x
Reference in New Issue
Block a user