Skip to content

Commit 95b55f3

Browse files
committed
Schedule ping by default for remote clusters
When we connect to remote clusters, there may be a few more routers/firewalls in-between compared to when we connect to nodes in the same cluster. We've experienced cases where firewalls drop connections completely and keep-alives seem not to be enough, or they are not properly configured. With this commit we enable application-level pings by default every 5 seconds from CCS nodes to the selected remote nodes. We also add a setting called `cluster.remote.ping_schedule` that allows to change the interval and potentially disable application-level pings, similar to `transport.ping_schedule` but the new setting only affects connections made to remote clusters. Relates to elastic#34405
1 parent edfdac5 commit 95b55f3

File tree

6 files changed

+89
-22
lines changed

6 files changed

+89
-22
lines changed

docs/reference/modules/remote-clusters.asciidoc

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -129,6 +129,12 @@ PUT _cluster/settings
129129
The time to wait for remote connections to be established when the node
130130
starts. The default is `30s`.
131131

132+
`cluster.remote.ping_schedule`::
133+
134+
Schedule a regular application-level ping message to ensure that transport
135+
connections to nodes belonging to remote clusters are kept alive. Defaults
136+
to `5s`, it can be set to `-1` to disable pings.
137+
132138
`cluster.remote.node.attr`::
133139

134140
A node attribute to filter out nodes that are eligible as a gateway node in

docs/reference/modules/transport.asciidoc

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -46,10 +46,10 @@ between all nodes. Defaults to `false`.
4646

4747
|`transport.ping_schedule` | Schedule a regular application-level ping message
4848
to ensure that transport connections between nodes are kept alive. Defaults to
49-
`5s` in the transport client and `-1` (disabled) elsewhere. It is preferable to
50-
correctly configure TCP keep-alives instead of using this feature, because TCP
51-
keep-alives apply to all kinds of long-lived connection and not just to
52-
transport connections.
49+
`5s` in the transport client and connections to remote clusters, and `-1`
50+
(disabled) elsewhere. It is preferable to correctly configure TCP keep-alives
51+
instead of using this feature, because TCP keep-alives apply to all kinds of
52+
long-lived connections and not just to transport connections.
5353

5454
|=======================================================================
5555

server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -292,6 +292,7 @@ public void apply(Settings value, Settings current, Settings previous) {
292292
RemoteClusterService.SEARCH_REMOTE_NODE_ATTRIBUTE,
293293
RemoteClusterService.ENABLE_REMOTE_CLUSTERS,
294294
RemoteClusterService.SEARCH_ENABLE_REMOTE_CLUSTERS,
295+
RemoteClusterService.REMOTE_PING_SCHEDULE,
295296
TransportService.TRACE_LOG_EXCLUDE_SETTING,
296297
TransportService.TRACE_LOG_INCLUDE_SETTING,
297298
TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING,

server/src/main/java/org/elasticsearch/transport/ConnectionManager.java

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -67,12 +67,15 @@ public class ConnectionManager implements Closeable {
6767
private final DelegatingNodeConnectionListener connectionListener = new DelegatingNodeConnectionListener();
6868

6969
public ConnectionManager(Settings settings, Transport transport, ThreadPool threadPool) {
70+
this(settings, transport, threadPool, TcpTransport.PING_SCHEDULE.get(settings));
71+
}
72+
73+
public ConnectionManager(Settings settings, Transport transport, ThreadPool threadPool, TimeValue pingSchedule) {
7074
this.transport = transport;
7175
this.threadPool = threadPool;
72-
this.pingSchedule = TcpTransport.PING_SCHEDULE.get(settings);
76+
this.pingSchedule = pingSchedule;
7377
this.defaultProfile = buildDefaultConnectionProfile(settings);
7478
this.lifecycle.moveToStarted();
75-
7679
if (pingSchedule.millis() > 0) {
7780
threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, new ScheduledPing());
7881
}
@@ -287,6 +290,10 @@ public void onFailure(Exception e) {
287290
}
288291
}
289292

293+
public TimeValue getPingSchedule() {
294+
return pingSchedule;
295+
}
296+
290297
private static final class DelegatingNodeConnectionListener implements TransportConnectionListener {
291298

292299
private final CopyOnWriteArrayList<TransportConnectionListener> listeners = new CopyOnWriteArrayList<>();

server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java

Lines changed: 16 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,10 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
7171
assert Version.CURRENT.major < 8;
7272
}
7373

74+
//the default here (5s) differs from the default in TcpTransport.PING_SCHEDULE (which is -1, hence disabled)
75+
public static final Setting<TimeValue> REMOTE_PING_SCHEDULE =
76+
Setting.timeSetting("cluster.remote.ping_schedule", TimeValue.timeValueSeconds(5), Setting.Property.NodeScope);
77+
7478
public static final Setting<Integer> SEARCH_REMOTE_CONNECTIONS_PER_CLUSTER =
7579
Setting.intSetting("search.remote.connections_per_cluster", 3, 1, Setting.Property.NodeScope, Setting.Property.Deprecated);
7680

@@ -211,9 +215,10 @@ private synchronized void updateRemoteClusters(Map<String, Tuple<String, List<Su
211215
}
212216

213217
if (remote == null) { // this is a new cluster we have to add a new representation
214-
remote = new RemoteClusterConnection(settings, entry.getKey(), seedList, transportService,
215-
new ConnectionManager(settings, transportService.transport, transportService.threadPool), numRemoteConnections,
216-
getNodePredicate(settings), proxyAddress);
218+
ConnectionManager connectionManager = new ConnectionManager(settings, transportService.transport,
219+
transportService.threadPool, REMOTE_PING_SCHEDULE.get(settings));
220+
remote = new RemoteClusterConnection(settings, entry.getKey(), seedList, transportService, connectionManager,
221+
numRemoteConnections, getNodePredicate(settings), proxyAddress);
217222
remoteClusters.put(entry.getKey(), remote);
218223
}
219224

@@ -340,31 +345,27 @@ public void onFailure(Exception e) {
340345
* @throws IllegalArgumentException if the remote cluster is unknown
341346
*/
342347
public Transport.Connection getConnection(DiscoveryNode node, String cluster) {
343-
RemoteClusterConnection connection = remoteClusters.get(cluster);
344-
if (connection == null) {
345-
throw new IllegalArgumentException("no such remote cluster: " + cluster);
346-
}
347-
return connection.getConnection(node);
348+
return getRemoteClusterConnection(cluster).getConnection(node);
348349
}
349350

350351
/**
351352
* Ensures that the given cluster alias is connected. If the cluster is connected this operation
352353
* will invoke the listener immediately.
353354
*/
354-
public void ensureConnected(String clusterAlias, ActionListener<Void> listener) {
355-
RemoteClusterConnection remoteClusterConnection = remoteClusters.get(clusterAlias);
356-
if (remoteClusterConnection == null) {
357-
throw new IllegalArgumentException("no such remote cluster: " + clusterAlias);
358-
}
359-
remoteClusterConnection.ensureConnected(listener);
355+
void ensureConnected(String clusterAlias, ActionListener<Void> listener) {
356+
getRemoteClusterConnection(clusterAlias).ensureConnected(listener);
360357
}
361358

362359
public Transport.Connection getConnection(String cluster) {
360+
return getRemoteClusterConnection(cluster).getConnection();
361+
}
362+
363+
RemoteClusterConnection getRemoteClusterConnection(String cluster) {
363364
RemoteClusterConnection connection = remoteClusters.get(cluster);
364365
if (connection == null) {
365366
throw new IllegalArgumentException("no such remote cluster: " + cluster);
366367
}
367-
return connection.getConnection();
368+
return connection;
368369
}
369370

370371
@Override
@@ -386,7 +387,6 @@ synchronized void updateSkipUnavailable(String clusterAlias, Boolean skipUnavail
386387
}
387388
}
388389

389-
390390
@Override
391391
protected void updateRemoteCluster(String clusterAlias, List<String> addresses, String proxyAddress) {
392392
updateRemoteCluster(clusterAlias, addresses, proxyAddress, ActionListener.wrap((x) -> {}, (x) -> {}));

server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java

Lines changed: 53 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.elasticsearch.common.settings.ClusterSettings;
3232
import org.elasticsearch.common.settings.Settings;
3333
import org.elasticsearch.common.transport.TransportAddress;
34+
import org.elasticsearch.common.unit.TimeValue;
3435
import org.elasticsearch.core.internal.io.IOUtils;
3536
import org.elasticsearch.rest.RestStatus;
3637
import org.elasticsearch.test.ESTestCase;
@@ -97,6 +98,7 @@ public void testSettingsAreRegistered() {
9798
assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER));
9899
assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING));
99100
assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_NODE_ATTRIBUTE));
101+
assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_PING_SCHEDULE));
100102
}
101103

102104
public void testRemoteClusterSeedSetting() {
@@ -337,6 +339,57 @@ public void testIncrementallyAddClusters() throws IOException {
337339
}
338340
}
339341

342+
public void testDefaultPingSchedule() throws IOException {
343+
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
344+
try (MockTransportService seedTransport = startTransport("cluster_1_node", knownNodes, Version.CURRENT)) {
345+
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
346+
knownNodes.add(seedTransport.getLocalDiscoNode());
347+
348+
try (MockTransportService transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool,
349+
null)) {
350+
transportService.start();
351+
transportService.acceptIncomingRequests();
352+
try (RemoteClusterService service = new RemoteClusterService(Settings.EMPTY, transportService)) {
353+
assertFalse(service.isCrossClusterSearchEnabled());
354+
service.initializeRemoteClusters();
355+
assertFalse(service.isCrossClusterSearchEnabled());
356+
service.updateRemoteCluster("cluster_1", Collections.singletonList(seedNode.getAddress().toString()), null);
357+
assertTrue(service.isCrossClusterSearchEnabled());
358+
assertTrue(service.isRemoteClusterRegistered("cluster_1"));
359+
RemoteClusterConnection remoteClusterConnection = service.getRemoteClusterConnection("cluster_1");
360+
assertEquals(TimeValue.timeValueSeconds(5), remoteClusterConnection.getConnectionManager().getPingSchedule());
361+
}
362+
}
363+
}
364+
}
365+
366+
public void testCustomPingSchedule() throws IOException {
367+
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
368+
try (MockTransportService seedTransport = startTransport("cluster_1_node", knownNodes, Version.CURRENT)) {
369+
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
370+
knownNodes.add(seedTransport.getLocalDiscoNode());
371+
372+
try (MockTransportService transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool,
373+
null)) {
374+
transportService.start();
375+
transportService.acceptIncomingRequests();
376+
Settings.Builder builder = Settings.builder();
377+
TimeValue pingSchedule = randomBoolean() ? TimeValue.MINUS_ONE : TimeValue.timeValueSeconds(randomIntBetween(1, 10));
378+
builder.put("cluster.remote.ping_schedule", pingSchedule);
379+
try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) {
380+
assertFalse(service.isCrossClusterSearchEnabled());
381+
service.initializeRemoteClusters();
382+
assertFalse(service.isCrossClusterSearchEnabled());
383+
service.updateRemoteCluster("cluster_1", Collections.singletonList(seedNode.getAddress().toString()), null);
384+
assertTrue(service.isCrossClusterSearchEnabled());
385+
assertTrue(service.isRemoteClusterRegistered("cluster_1"));
386+
RemoteClusterConnection remoteClusterConnection = service.getRemoteClusterConnection("cluster_1");
387+
assertEquals(pingSchedule, remoteClusterConnection.getConnectionManager().getPingSchedule());
388+
}
389+
}
390+
}
391+
}
392+
340393
public void testRemoteNodeAttribute() throws IOException, InterruptedException {
341394
final Settings settings =
342395
Settings.builder().put("cluster.remote.node.attr", "gateway").build();

0 commit comments

Comments
 (0)