mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-06-28 09:28:55 -04:00
transport: edit TransportConnectionListener for close exceptions (#129015)
The TransportConnectionListener interface has previously included the Transport.Connection being closed and unregistered in its onNodeDisconnected callback. This is not in use, and can be removed as it is also available in the onConnectionClosed callback. It is being replaced with a Nullable exception that caused the close. This is being used in pending work (ES-11448) to differentiate network issues from node restarts. Closes ES-12007
This commit is contained in:
parent
aceaf23130
commit
ee716f11b9
11 changed files with 42 additions and 23 deletions
|
@ -26,6 +26,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
|||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.core.CheckedRunnable;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.monitor.NodeHealthService;
|
||||
import org.elasticsearch.monitor.StatusInfo;
|
||||
|
@ -33,7 +34,6 @@ import org.elasticsearch.threadpool.ThreadPool.Names;
|
|||
import org.elasticsearch.transport.AbstractTransportRequest;
|
||||
import org.elasticsearch.transport.ConnectTransportException;
|
||||
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
|
@ -137,7 +137,7 @@ public final class FollowersChecker {
|
|||
);
|
||||
transportService.addConnectionListener(new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
handleDisconnectedNode(node);
|
||||
}
|
||||
});
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.elasticsearch.transport.AbstractTransportRequest;
|
|||
import org.elasticsearch.transport.ConnectTransportException;
|
||||
import org.elasticsearch.transport.NodeDisconnectedException;
|
||||
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
|
@ -124,7 +123,7 @@ public class LeaderChecker {
|
|||
|
||||
transportService.addConnectionListener(new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
handleDisconnectedNode(node);
|
||||
}
|
||||
});
|
||||
|
|
|
@ -229,11 +229,26 @@ public class ClusterConnectionManager implements ConnectionManager {
|
|||
try {
|
||||
connectionListener.onNodeConnected(node, conn);
|
||||
} finally {
|
||||
conn.addCloseListener(ActionListener.running(() -> {
|
||||
connectedNodes.remove(node, conn);
|
||||
connectionListener.onNodeDisconnected(node, conn);
|
||||
managerRefs.decRef();
|
||||
}));
|
||||
conn.addCloseListener(new ActionListener<Void>() {
|
||||
@Override
|
||||
public void onResponse(Void ignored) {
|
||||
handleClose(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
handleClose(e);
|
||||
}
|
||||
|
||||
void handleClose(@Nullable Exception e) {
|
||||
connectedNodes.remove(node, conn);
|
||||
try {
|
||||
connectionListener.onNodeDisconnected(node, e);
|
||||
} finally {
|
||||
managerRefs.decRef();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
conn.addCloseListener(ActionListener.running(() -> {
|
||||
if (connectingRefCounter.hasReferences() == false) {
|
||||
|
|
|
@ -60,9 +60,9 @@ public interface ConnectionManager extends Closeable {
|
|||
private final CopyOnWriteArrayList<TransportConnectionListener> listeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode key, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode key, @Nullable Exception closeException) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onNodeDisconnected(key, connection);
|
||||
listener.onNodeDisconnected(key, closeException);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -61,7 +61,7 @@ public class RemoteConnectionManager implements ConnectionManager {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
removeConnectedNode(node);
|
||||
}
|
||||
});
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.elasticsearch.common.settings.Setting;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
|
@ -339,7 +340,7 @@ public abstract class RemoteConnectionStrategy implements TransportConnectionLis
|
|||
protected abstract ConnectionStrategy strategyType();
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
if (shouldOpenMoreConnections()) {
|
||||
// try to reconnect and fill up the slot of the disconnected node
|
||||
connect(
|
||||
|
|
|
@ -10,6 +10,7 @@
|
|||
package org.elasticsearch.transport;
|
||||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
|
||||
/**
|
||||
* A listener interface that allows to react on transport events. All methods may be
|
||||
|
@ -38,5 +39,5 @@ public interface TransportConnectionListener {
|
|||
/**
|
||||
* Called once a node connection is closed and unregistered.
|
||||
*/
|
||||
default void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {}
|
||||
default void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.elasticsearch.common.settings.ClusterSettings;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.core.Tuple;
|
||||
import org.elasticsearch.index.Index;
|
||||
|
@ -831,7 +832,7 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
CountDownLatch disconnectedLatch = new CountDownLatch(numDisconnectedClusters);
|
||||
RemoteClusterServiceTests.addConnectionListener(remoteClusterService, new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
if (disconnectedNodes.remove(node)) {
|
||||
disconnectedLatch.countDown();
|
||||
}
|
||||
|
@ -1134,7 +1135,7 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
CountDownLatch disconnectedLatch = new CountDownLatch(numDisconnectedClusters);
|
||||
RemoteClusterServiceTests.addConnectionListener(remoteClusterService, new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
if (disconnectedNodes.remove(node)) {
|
||||
disconnectedLatch.countDown();
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.elasticsearch.common.transport.TransportAddress;
|
|||
import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue;
|
||||
import org.elasticsearch.core.AbstractRefCounted;
|
||||
import org.elasticsearch.core.CheckedRunnable;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.core.RefCounted;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
@ -251,7 +252,7 @@ public class NodeConnectionsServiceTests extends ESTestCase {
|
|||
final AtomicReference<ActionListener<DiscoveryNode>> disconnectListenerRef = new AtomicReference<>();
|
||||
transportService.addConnectionListener(new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
final ActionListener<DiscoveryNode> disconnectListener = disconnectListenerRef.getAndSet(null);
|
||||
if (disconnectListener != null) {
|
||||
disconnectListener.onResponse(node);
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
|||
import org.elasticsearch.common.util.concurrent.RunOnce;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.core.AbstractRefCounted;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.core.Releasable;
|
||||
import org.elasticsearch.core.Releasables;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
|
@ -101,7 +102,7 @@ public class ClusterConnectionManagerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
nodeDisconnectedCount.incrementAndGet();
|
||||
}
|
||||
});
|
||||
|
@ -658,7 +659,7 @@ public class ClusterConnectionManagerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
nodeDisconnectedCount.incrementAndGet();
|
||||
}
|
||||
});
|
||||
|
@ -698,7 +699,7 @@ public class ClusterConnectionManagerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
nodeDisconnectedCount.incrementAndGet();
|
||||
}
|
||||
});
|
||||
|
|
|
@ -202,7 +202,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
fail("disconnect should not be called " + node);
|
||||
}
|
||||
};
|
||||
|
@ -924,7 +924,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
latch.countDown();
|
||||
}
|
||||
};
|
||||
|
@ -2118,7 +2118,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
|
||||
public void onNodeDisconnected(DiscoveryNode node, @Nullable Exception closeException) {
|
||||
fail("disconnect should not be called " + node);
|
||||
}
|
||||
};
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue