Skip to content

Commit 43e2382

Browse files
committed
Connect to new nodes concurrently (#22984)
When a node receives a new cluster state from the master, it opens up connections to any new node in the cluster state. That has always been done serially on the cluster state thread but it has been a long standing TODO to do this concurrently, which is done by this PR. This is spin off of #22828, where an extra handshake is done whenever connecting to a node, which may slow down connecting. Also, the handshake is done in a blocking fashion which triggers assertions w.r.t blocking requests on the cluster state thread. Instead of adding an exception, I opted to implement concurrent connections which both side steps the assertion and compensates for the extra handshake.
1 parent 360e76d commit 43e2382

File tree

4 files changed

+57
-29
lines changed

4 files changed

+57
-29
lines changed

core/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java

Lines changed: 42 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import org.apache.logging.log4j.message.ParameterizedMessage;
2222
import org.apache.logging.log4j.util.Supplier;
2323
import org.elasticsearch.cluster.node.DiscoveryNode;
24+
import org.elasticsearch.cluster.node.DiscoveryNodes;
2425
import org.elasticsearch.common.component.AbstractLifecycleComponent;
2526
import org.elasticsearch.common.inject.Inject;
2627
import org.elasticsearch.common.lease.Releasable;
@@ -37,9 +38,9 @@
3738
import org.elasticsearch.transport.TransportService;
3839

3940
import java.util.HashSet;
40-
import java.util.List;
4141
import java.util.Set;
4242
import java.util.concurrent.ConcurrentMap;
43+
import java.util.concurrent.CountDownLatch;
4344
import java.util.concurrent.ScheduledFuture;
4445

4546
import static org.elasticsearch.common.settings.Setting.Property;
@@ -78,21 +79,53 @@ public NodeConnectionsService(Settings settings, ThreadPool threadPool, Transpor
7879
this.reconnectInterval = NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING.get(settings);
7980
}
8081

81-
public void connectToNodes(Iterable<DiscoveryNode> discoveryNodes) {
82-
83-
// TODO: do this in parallel (and wait)
82+
public void connectToNodes(DiscoveryNodes discoveryNodes) {
83+
CountDownLatch latch = new CountDownLatch(discoveryNodes.getSize());
8484
for (final DiscoveryNode node : discoveryNodes) {
85+
final boolean connected;
8586
try (Releasable ignored = nodeLocks.acquire(node)) {
8687
nodes.putIfAbsent(node, 0);
87-
validateNodeConnected(node);
88+
connected = transportService.nodeConnected(node);
89+
}
90+
if (connected) {
91+
latch.countDown();
92+
} else {
93+
// spawn to another thread to do in parallel
94+
threadPool.executor(ThreadPool.Names.MANAGEMENT).execute(new AbstractRunnable() {
95+
@Override
96+
public void onFailure(Exception e) {
97+
// both errors and rejections are logged here. the service
98+
// will try again after `cluster.nodes.reconnect_interval` on all nodes but the current master.
99+
// On the master, node fault detection will remove these nodes from the cluster as their are not
100+
// connected. Note that it is very rare that we end up here on the master.
101+
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to connect to {}", node), e);
102+
}
103+
104+
@Override
105+
protected void doRun() throws Exception {
106+
try (Releasable ignored = nodeLocks.acquire(node)) {
107+
validateAndConnectIfNeeded(node);
108+
}
109+
}
110+
111+
@Override
112+
public void onAfter() {
113+
latch.countDown();
114+
}
115+
});
88116
}
89117
}
118+
try {
119+
latch.await();
120+
} catch (InterruptedException e) {
121+
Thread.currentThread().interrupt();
122+
}
90123
}
91124

92125
/**
93126
* Disconnects from all nodes except the ones provided as parameter
94127
*/
95-
public void disconnectFromNodesExcept(Iterable<DiscoveryNode> nodesToKeep) {
128+
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
96129
Set<DiscoveryNode> currentNodes = new HashSet<>(nodes.keySet());
97130
for (DiscoveryNode node : nodesToKeep) {
98131
currentNodes.remove(node);
@@ -110,8 +143,8 @@ public void disconnectFromNodesExcept(Iterable<DiscoveryNode> nodesToKeep) {
110143
}
111144
}
112145

113-
void validateNodeConnected(DiscoveryNode node) {
114-
assert nodeLocks.isHeldByCurrentThread(node) : "validateNodeConnected must be called under lock";
146+
void validateAndConnectIfNeeded(DiscoveryNode node) {
147+
assert nodeLocks.isHeldByCurrentThread(node) : "validateAndConnectIfNeeded must be called under lock";
115148
if (lifecycle.stoppedOrClosed() ||
116149
nodes.containsKey(node) == false) { // we double check existence of node since connectToNode might take time...
117150
// nothing to do
@@ -147,7 +180,7 @@ public void onFailure(Exception e) {
147180
protected void doRun() {
148181
for (DiscoveryNode node : nodes.keySet()) {
149182
try (Releasable ignored = nodeLocks.acquire(node)) {
150-
validateNodeConnected(node);
183+
validateAndConnectIfNeeded(node);
151184
}
152185
}
153186
}

core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java

Lines changed: 9 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.elasticsearch.common.transport.TransportAddress;
3232
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
3333
import org.elasticsearch.test.ESTestCase;
34+
import org.elasticsearch.threadpool.TestThreadPool;
3435
import org.elasticsearch.threadpool.ThreadPool;
3536
import org.elasticsearch.transport.ConnectTransportException;
3637
import org.elasticsearch.transport.ConnectionProfile;
@@ -41,7 +42,6 @@
4142
import org.elasticsearch.transport.TransportService;
4243
import org.elasticsearch.transport.TransportServiceAdapter;
4344
import org.junit.After;
44-
import org.junit.AfterClass;
4545
import org.junit.Before;
4646

4747
import java.io.IOException;
@@ -60,7 +60,7 @@
6060

6161
public class NodeConnectionsServiceTests extends ESTestCase {
6262

63-
private static ThreadPool THREAD_POOL;
63+
private ThreadPool threadPool;
6464
private MockTransport transport;
6565
private TransportService transportService;
6666

@@ -84,7 +84,7 @@ private ClusterState clusterStateFromNodes(List<DiscoveryNode> nodes) {
8484

8585
public void testConnectAndDisconnect() {
8686
List<DiscoveryNode> nodes = generateNodes();
87-
NodeConnectionsService service = new NodeConnectionsService(Settings.EMPTY, THREAD_POOL, transportService);
87+
NodeConnectionsService service = new NodeConnectionsService(Settings.EMPTY, threadPool, transportService);
8888

8989
ClusterState current = clusterStateFromNodes(Collections.emptyList());
9090
ClusterChangedEvent event = new ClusterChangedEvent("test", clusterStateFromNodes(randomSubsetOf(nodes)), current);
@@ -108,14 +108,14 @@ public void testConnectAndDisconnect() {
108108

109109
public void testReconnect() {
110110
List<DiscoveryNode> nodes = generateNodes();
111-
NodeConnectionsService service = new NodeConnectionsService(Settings.EMPTY, THREAD_POOL, transportService);
111+
NodeConnectionsService service = new NodeConnectionsService(Settings.EMPTY, threadPool, transportService);
112112

113113
ClusterState current = clusterStateFromNodes(Collections.emptyList());
114114
ClusterChangedEvent event = new ClusterChangedEvent("test", clusterStateFromNodes(randomSubsetOf(nodes)), current);
115115

116116
transport.randomConnectionExceptions = true;
117117

118-
service.connectToNodes(event.nodesDelta().addedNodes());
118+
service.connectToNodes(event.state().nodes());
119119

120120
for (int i = 0; i < 3; i++) {
121121
// simulate disconnects
@@ -152,8 +152,9 @@ private void assertNotConnected(Iterable<DiscoveryNode> nodes) {
152152
@Before
153153
public void setUp() throws Exception {
154154
super.setUp();
155+
this.threadPool = new TestThreadPool(getClass().getName());
155156
this.transport = new MockTransport();
156-
transportService = new TransportService(Settings.EMPTY, transport, THREAD_POOL, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
157+
transportService = new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
157158
boundAddress -> DiscoveryNode.createLocal(Settings.EMPTY, LocalTransportAddress.buildUnique(), UUIDs.randomBase64UUID()), null);
158159
transportService.start();
159160
transportService.acceptIncomingRequests();
@@ -163,16 +164,11 @@ public void setUp() throws Exception {
163164
@After
164165
public void tearDown() throws Exception {
165166
transportService.stop();
167+
ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS);
168+
threadPool = null;
166169
super.tearDown();
167170
}
168171

169-
@AfterClass
170-
public static void stopThreadPool() {
171-
ThreadPool.terminate(THREAD_POOL, 30, TimeUnit.SECONDS);
172-
THREAD_POOL = null;
173-
}
174-
175-
176172
final class MockTransport implements Transport {
177173
private final AtomicLong requestId = new AtomicLong();
178174
Set<DiscoveryNode> connectedNodes = ConcurrentCollections.newConcurrentSet();

core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -126,12 +126,12 @@ TimedClusterService createTimedClusterService(boolean makeMaster) throws Interru
126126
threadPool, () -> new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT));
127127
timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
128128
@Override
129-
public void connectToNodes(Iterable<DiscoveryNode> discoveryNodes) {
129+
public void connectToNodes(DiscoveryNodes discoveryNodes) {
130130
// skip
131131
}
132132

133133
@Override
134-
public void disconnectFromNodesExcept(Iterable<DiscoveryNode> nodesToKeep) {
134+
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
135135
// skip
136136
}
137137
});
@@ -1058,12 +1058,12 @@ public void testDisconnectFromNewlyAddedNodesIfClusterStatePublishingFails() thr
10581058
Set<DiscoveryNode> currentNodes = new HashSet<>();
10591059
timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
10601060
@Override
1061-
public void connectToNodes(Iterable<DiscoveryNode> discoveryNodes) {
1061+
public void connectToNodes(DiscoveryNodes discoveryNodes) {
10621062
discoveryNodes.forEach(currentNodes::add);
10631063
}
10641064

10651065
@Override
1066-
public void disconnectFromNodesExcept(Iterable<DiscoveryNode> nodesToKeep) {
1066+
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
10671067
Set<DiscoveryNode> nodeSet = new HashSet<>();
10681068
nodesToKeep.iterator().forEachRemaining(nodeSet::add);
10691069
currentNodes.removeIf(node -> nodeSet.contains(node) == false);

test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,6 @@
3535
import java.util.Arrays;
3636
import java.util.Collections;
3737
import java.util.HashSet;
38-
import java.util.List;
3938
import java.util.concurrent.CountDownLatch;
4039

4140
import static junit.framework.TestCase.fail;
@@ -54,12 +53,12 @@ public static ClusterService createClusterService(ThreadPool threadPool, Discove
5453
threadPool, () -> localNode);
5554
clusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
5655
@Override
57-
public void connectToNodes(Iterable<DiscoveryNode> discoveryNodes) {
56+
public void connectToNodes(DiscoveryNodes discoveryNodes) {
5857
// skip
5958
}
6059

6160
@Override
62-
public void disconnectFromNodesExcept(Iterable<DiscoveryNode> nodesToKeep) {
61+
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
6362
// skip
6463
}
6564
});

0 commit comments

Comments
 (0)