|
@@ -22,6 +22,7 @@ import org.apache.lucene.store.AlreadyClosedException;
|
|
|
import org.elasticsearch.Build;
|
|
import org.elasticsearch.Build;
|
|
|
import org.elasticsearch.Version;
|
|
import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.action.ActionListener;
|
|
import org.elasticsearch.action.ActionListener;
|
|
|
|
|
+import org.elasticsearch.action.LatchedActionListener;
|
|
|
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
|
|
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
|
|
|
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoAction;
|
|
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoAction;
|
|
|
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
|
|
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
|
|
@@ -730,4 +731,58 @@ public class RemoteClusterConnectionTests extends ESTestCase {
|
|
|
}
|
|
}
|
|
|
return statsRef.get();
|
|
return statsRef.get();
|
|
|
}
|
|
}
|
|
|
|
|
+
|
|
|
|
|
+ public void testEnsureConnected() throws IOException, InterruptedException {
|
|
|
|
|
+ List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
|
|
|
|
|
+ try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
|
|
|
|
|
+ MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
|
|
|
|
|
+ DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
|
|
|
|
|
+ DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
|
|
|
|
|
+ knownNodes.add(seedTransport.getLocalDiscoNode());
|
|
|
|
|
+ knownNodes.add(discoverableTransport.getLocalDiscoNode());
|
|
|
|
|
+ Collections.shuffle(knownNodes, random());
|
|
|
|
|
+
|
|
|
|
|
+ try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
|
|
|
|
|
+ service.start();
|
|
|
|
|
+ service.acceptIncomingRequests();
|
|
|
|
|
+ try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
|
|
|
|
|
+ Arrays.asList(seedNode), service, Integer.MAX_VALUE, n -> true)) {
|
|
|
|
|
+ assertFalse(service.nodeConnected(seedNode));
|
|
|
|
|
+ assertFalse(service.nodeConnected(discoverableNode));
|
|
|
|
|
+ assertTrue(connection.assertNoRunningConnections());
|
|
|
|
|
+ CountDownLatch latch = new CountDownLatch(1);
|
|
|
|
|
+ connection.ensureConnected(new LatchedActionListener<>(new ActionListener<Void>() {
|
|
|
|
|
+ @Override
|
|
|
|
|
+ public void onResponse(Void aVoid) {
|
|
|
|
|
+ }
|
|
|
|
|
+
|
|
|
|
|
+ @Override
|
|
|
|
|
+ public void onFailure(Exception e) {
|
|
|
|
|
+ throw new AssertionError(e);
|
|
|
|
|
+ }
|
|
|
|
|
+ }, latch));
|
|
|
|
|
+ latch.await();
|
|
|
|
|
+ assertTrue(service.nodeConnected(seedNode));
|
|
|
|
|
+ assertTrue(service.nodeConnected(discoverableNode));
|
|
|
|
|
+ assertTrue(connection.assertNoRunningConnections());
|
|
|
|
|
+
|
|
|
|
|
+ // exec again we are already connected
|
|
|
|
|
+ connection.ensureConnected(new LatchedActionListener<>(new ActionListener<Void>() {
|
|
|
|
|
+ @Override
|
|
|
|
|
+ public void onResponse(Void aVoid) {
|
|
|
|
|
+ }
|
|
|
|
|
+
|
|
|
|
|
+ @Override
|
|
|
|
|
+ public void onFailure(Exception e) {
|
|
|
|
|
+ throw new AssertionError(e);
|
|
|
|
|
+ }
|
|
|
|
|
+ }, latch));
|
|
|
|
|
+ latch.await();
|
|
|
|
|
+ assertTrue(service.nodeConnected(seedNode));
|
|
|
|
|
+ assertTrue(service.nodeConnected(discoverableNode));
|
|
|
|
|
+ assertTrue(connection.assertNoRunningConnections());
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
}
|
|
}
|