35
35
import org .elasticsearch .cluster .node .DiscoveryNode ;
36
36
import org .elasticsearch .cluster .node .DiscoveryNodes ;
37
37
import org .elasticsearch .common .component .AbstractComponent ;
38
+ import org .elasticsearch .common .io .stream .StreamInput ;
38
39
import org .elasticsearch .common .settings .Settings ;
39
40
import org .elasticsearch .common .transport .TransportAddress ;
40
41
import org .elasticsearch .common .unit .TimeValue ;
80
81
final class RemoteClusterConnection extends AbstractComponent implements TransportConnectionListener , Closeable {
81
82
82
83
private final TransportService transportService ;
84
+ private final ConnectionManager connectionManager ;
83
85
private final ConnectionProfile remoteProfile ;
84
86
private final ConnectedNodes connectedNodes ;
85
87
private final String clusterAlias ;
86
88
private final int maxNumRemoteConnections ;
87
89
private final Predicate <DiscoveryNode > nodePredicate ;
90
+ private final ThreadPool threadPool ;
88
91
private volatile List <Supplier <DiscoveryNode >> seedNodes ;
89
92
private volatile boolean skipUnavailable ;
90
93
private final ConnectHandler connectHandler ;
91
94
private SetOnce <ClusterName > remoteClusterName = new SetOnce <>();
92
- private final ClusterName localClusterName ;
93
95
94
96
/**
95
97
* Creates a new {@link RemoteClusterConnection}
96
98
* @param settings the nodes settings object
97
99
* @param clusterAlias the configured alias of the cluster to connect to
98
100
* @param seedNodes a list of seed nodes to discover eligible nodes from
99
101
* @param transportService the local nodes transport service
102
+ * @param connectionManager the connection manager to use for this remote connection
100
103
* @param maxNumRemoteConnections the maximum number of connections to the remote cluster
101
104
* @param nodePredicate a predicate to filter eligible remote nodes to connect to
102
105
*/
103
106
RemoteClusterConnection (Settings settings , String clusterAlias , List <Supplier <DiscoveryNode >> seedNodes ,
104
- TransportService transportService , int maxNumRemoteConnections , Predicate <DiscoveryNode > nodePredicate ) {
107
+ TransportService transportService , ConnectionManager connectionManager , int maxNumRemoteConnections ,
108
+ Predicate <DiscoveryNode > nodePredicate ) {
105
109
super (settings );
106
- this .localClusterName = ClusterName .CLUSTER_NAME_SETTING .get (settings );
107
110
this .transportService = transportService ;
108
111
this .maxNumRemoteConnections = maxNumRemoteConnections ;
109
112
this .nodePredicate = nodePredicate ;
@@ -122,7 +125,11 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo
122
125
this .skipUnavailable = RemoteClusterService .REMOTE_CLUSTER_SKIP_UNAVAILABLE
123
126
.getConcreteSettingForNamespace (clusterAlias ).get (settings );
124
127
this .connectHandler = new ConnectHandler ();
125
- transportService .addConnectionListener (this );
128
+ this .threadPool = transportService .threadPool ;
129
+ this .connectionManager = connectionManager ;
130
+ connectionManager .addListener (this );
131
+ // we register the transport service here as a listener to make sure we notify handlers on disconnect etc.
132
+ connectionManager .addListener (transportService );
126
133
}
127
134
128
135
/**
@@ -183,8 +190,9 @@ public void ensureConnected(ActionListener<Void> voidActionListener) {
183
190
184
191
private void fetchShardsInternal (ClusterSearchShardsRequest searchShardsRequest ,
185
192
final ActionListener <ClusterSearchShardsResponse > listener ) {
186
- final DiscoveryNode node = connectedNodes .getAny ();
187
- transportService .sendRequest (node , ClusterSearchShardsAction .NAME , searchShardsRequest ,
193
+ final DiscoveryNode node = getAnyConnectedNode ();
194
+ Transport .Connection connection = connectionManager .getConnection (node );
195
+ transportService .sendRequest (connection , ClusterSearchShardsAction .NAME , searchShardsRequest , TransportRequestOptions .EMPTY ,
188
196
new TransportResponseHandler <ClusterSearchShardsResponse >() {
189
197
190
198
@ Override
@@ -219,12 +227,16 @@ void collectNodes(ActionListener<Function<String, DiscoveryNode>> listener) {
219
227
request .clear ();
220
228
request .nodes (true );
221
229
request .local (true ); // run this on the node that gets the request it's as good as any other
222
- final DiscoveryNode node = connectedNodes .getAny ();
223
- transportService .sendRequest (node , ClusterStateAction .NAME , request , TransportRequestOptions .EMPTY ,
230
+ final DiscoveryNode node = getAnyConnectedNode ();
231
+ Transport .Connection connection = connectionManager .getConnection (node );
232
+ transportService .sendRequest (connection , ClusterStateAction .NAME , request , TransportRequestOptions .EMPTY ,
224
233
new TransportResponseHandler <ClusterStateResponse >() {
234
+
225
235
@ Override
226
- public ClusterStateResponse newInstance () {
227
- return new ClusterStateResponse ();
236
+ public ClusterStateResponse read (StreamInput in ) throws IOException {
237
+ ClusterStateResponse response = new ClusterStateResponse ();
238
+ response .readFrom (in );
239
+ return response ;
228
240
}
229
241
230
242
@ Override
@@ -261,11 +273,11 @@ public String executor() {
261
273
* If such node is not connected, the returned connection will be a proxy connection that redirects to it.
262
274
*/
263
275
Transport .Connection getConnection (DiscoveryNode remoteClusterNode ) {
264
- if (transportService .nodeConnected (remoteClusterNode )) {
265
- return transportService .getConnection (remoteClusterNode );
276
+ if (connectionManager .nodeConnected (remoteClusterNode )) {
277
+ return connectionManager .getConnection (remoteClusterNode );
266
278
}
267
- DiscoveryNode discoveryNode = connectedNodes . getAny ();
268
- Transport .Connection connection = transportService .getConnection (discoveryNode );
279
+ DiscoveryNode discoveryNode = getAnyConnectedNode ();
280
+ Transport .Connection connection = connectionManager .getConnection (discoveryNode );
269
281
return new ProxyConnection (connection , remoteClusterNode );
270
282
}
271
283
@@ -317,33 +329,18 @@ public Version getVersion() {
317
329
}
318
330
319
331
Transport .Connection getConnection () {
320
- return transportService .getConnection (getAnyConnectedNode ());
332
+ return connectionManager .getConnection (getAnyConnectedNode ());
321
333
}
322
334
323
335
@ Override
324
336
public void close () throws IOException {
325
- connectHandler .close ();
337
+ IOUtils .close (connectHandler , connectionManager );
326
338
}
327
339
328
340
public boolean isClosed () {
329
341
return connectHandler .isClosed ();
330
342
}
331
343
332
- private ConnectionProfile getRemoteProfile (ClusterName name ) {
333
- // we can only compare the cluster name to make a decision if we should use a remote profile
334
- // we can't use a cluster UUID here since we could be connecting to that remote cluster before
335
- // the remote node has joined its cluster and have a cluster UUID. The fact that we just lose a
336
- // rather smallish optimization on the connection layer under certain situations where remote clusters
337
- // have the same name as the local one is minor here.
338
- // the alternative here is to complicate the remote infrastructure to also wait until we formed a cluster,
339
- // gained a cluster UUID and then start connecting etc. we rather use this simplification in order to maintain simplicity
340
- if (this .localClusterName .equals (name )) {
341
- return null ;
342
- } else {
343
- return remoteProfile ;
344
- }
345
- }
346
-
347
344
/**
348
345
* The connect handler manages node discovery and the actual connect to the remote cluster.
349
346
* There is at most one connect job running at any time. If such a connect job is triggered
@@ -387,7 +384,7 @@ private void connect(ActionListener<Void> connectListener, boolean forceRun) {
387
384
final boolean runConnect ;
388
385
final Collection <ActionListener <Void >> toNotify ;
389
386
final ActionListener <Void > listener = connectListener == null ? null :
390
- ContextPreservingActionListener .wrapPreservingContext (connectListener , transportService . getThreadPool () .getThreadContext ());
387
+ ContextPreservingActionListener .wrapPreservingContext (connectListener , threadPool .getThreadContext ());
391
388
synchronized (queue ) {
392
389
if (listener != null && queue .offer (listener ) == false ) {
393
390
listener .onFailure (new RejectedExecutionException ("connect queue is full" ));
@@ -415,7 +412,6 @@ private void connect(ActionListener<Void> connectListener, boolean forceRun) {
415
412
}
416
413
417
414
private void forkConnect (final Collection <ActionListener <Void >> toNotify ) {
418
- ThreadPool threadPool = transportService .getThreadPool ();
419
415
ExecutorService executor = threadPool .executor (ThreadPool .Names .MANAGEMENT );
420
416
executor .submit (new AbstractRunnable () {
421
417
@ Override
@@ -452,13 +448,13 @@ protected void doRun() {
452
448
maybeConnect ();
453
449
}
454
450
});
455
- collectRemoteNodes (seedNodes .iterator (), transportService , listener );
451
+ collectRemoteNodes (seedNodes .iterator (), transportService , connectionManager , listener );
456
452
}
457
453
});
458
454
}
459
455
460
456
private void collectRemoteNodes (Iterator <Supplier <DiscoveryNode >> seedNodes ,
461
- final TransportService transportService , ActionListener <Void > listener ) {
457
+ final TransportService transportService , final ConnectionManager manager , ActionListener <Void > listener ) {
462
458
if (Thread .currentThread ().isInterrupted ()) {
463
459
listener .onFailure (new InterruptedException ("remote connect thread got interrupted" ));
464
460
}
@@ -467,7 +463,7 @@ private void collectRemoteNodes(Iterator<Supplier<DiscoveryNode>> seedNodes,
467
463
cancellableThreads .executeIO (() -> {
468
464
final DiscoveryNode seedNode = seedNodes .next ().get ();
469
465
final TransportService .HandshakeResponse handshakeResponse ;
470
- Transport .Connection connection = transportService .openConnection (seedNode ,
466
+ Transport .Connection connection = manager .openConnection (seedNode ,
471
467
ConnectionProfile .buildSingleChannelProfile (TransportRequestOptions .Type .REG , null , null ));
472
468
boolean success = false ;
473
469
try {
@@ -482,7 +478,7 @@ private void collectRemoteNodes(Iterator<Supplier<DiscoveryNode>> seedNodes,
482
478
483
479
final DiscoveryNode handshakeNode = handshakeResponse .getDiscoveryNode ();
484
480
if (nodePredicate .test (handshakeNode ) && connectedNodes .size () < maxNumRemoteConnections ) {
485
- transportService .connectToNode (handshakeNode , getRemoteProfile ( handshakeResponse . getClusterName () ));
481
+ manager .connectToNode (handshakeNode , remoteProfile , transportService . connectionValidator ( handshakeNode ));
486
482
if (remoteClusterName .get () == null ) {
487
483
assert handshakeResponse .getClusterName ().value () != null ;
488
484
remoteClusterName .set (handshakeResponse .getClusterName ());
@@ -524,7 +520,7 @@ private void collectRemoteNodes(Iterator<Supplier<DiscoveryNode>> seedNodes,
524
520
// ISE if we fail the handshake with an version incompatible node
525
521
if (seedNodes .hasNext ()) {
526
522
logger .debug (() -> new ParameterizedMessage ("fetching nodes from external cluster {} failed" , clusterAlias ), ex );
527
- collectRemoteNodes (seedNodes , transportService , listener );
523
+ collectRemoteNodes (seedNodes , transportService , manager , listener );
528
524
} else {
529
525
listener .onFailure (ex );
530
526
}
@@ -552,7 +548,6 @@ final boolean isClosed() {
552
548
/* This class handles the _state response from the remote cluster when sniffing nodes to connect to */
553
549
private class SniffClusterStateResponseHandler implements TransportResponseHandler <ClusterStateResponse > {
554
550
555
- private final TransportService transportService ;
556
551
private final Transport .Connection connection ;
557
552
private final ActionListener <Void > listener ;
558
553
private final Iterator <Supplier <DiscoveryNode >> seedNodes ;
@@ -561,7 +556,6 @@ private class SniffClusterStateResponseHandler implements TransportResponseHandl
561
556
SniffClusterStateResponseHandler (TransportService transportService , Transport .Connection connection ,
562
557
ActionListener <Void > listener , Iterator <Supplier <DiscoveryNode >> seedNodes ,
563
558
CancellableThreads cancellableThreads ) {
564
- this .transportService = transportService ;
565
559
this .connection = connection ;
566
560
this .listener = listener ;
567
561
this .seedNodes = seedNodes ;
@@ -592,8 +586,8 @@ public void handleResponse(ClusterStateResponse response) {
592
586
for (DiscoveryNode node : nodesIter ) {
593
587
if (nodePredicate .test (node ) && connectedNodes .size () < maxNumRemoteConnections ) {
594
588
try {
595
- transportService .connectToNode (node , getRemoteProfile ( remoteClusterName . get ())); // noop if node is
596
- // connected
589
+ connectionManager .connectToNode (node , remoteProfile ,
590
+ transportService . connectionValidator ( node )); // noop if node is connected
597
591
connectedNodes .add (node );
598
592
} catch (ConnectTransportException | IllegalStateException ex ) {
599
593
// ISE if we fail the handshake with an version incompatible node
@@ -609,7 +603,7 @@ public void handleResponse(ClusterStateResponse response) {
609
603
listener .onFailure (ex ); // we got canceled - fail the listener and step out
610
604
} catch (Exception ex ) {
611
605
logger .warn (() -> new ParameterizedMessage ("fetching nodes from external cluster {} failed" , clusterAlias ), ex );
612
- collectRemoteNodes (seedNodes , transportService , listener );
606
+ collectRemoteNodes (seedNodes , transportService , connectionManager , listener );
613
607
}
614
608
}
615
609
@@ -620,7 +614,7 @@ public void handleException(TransportException exp) {
620
614
IOUtils .closeWhileHandlingException (connection );
621
615
} finally {
622
616
// once the connection is closed lets try the next node
623
- collectRemoteNodes (seedNodes , transportService , listener );
617
+ collectRemoteNodes (seedNodes , transportService , connectionManager , listener );
624
618
}
625
619
}
626
620
@@ -715,4 +709,8 @@ private synchronized void ensureIteratorAvailable() {
715
709
}
716
710
}
717
711
}
712
+
713
+ ConnectionManager getConnectionManager () {
714
+ return connectionManager ;
715
+ }
718
716
}
0 commit comments