22
22
import org .elasticsearch .action .ActionListener ;
23
23
import org .elasticsearch .action .OriginalIndices ;
24
24
import org .elasticsearch .action .admin .cluster .shards .ClusterSearchShardsGroup ;
25
+ import org .elasticsearch .action .admin .cluster .shards .ClusterSearchShardsRequest ;
25
26
import org .elasticsearch .action .admin .cluster .shards .ClusterSearchShardsResponse ;
26
27
import org .elasticsearch .action .support .ActionFilters ;
27
28
import org .elasticsearch .action .support .HandledTransportAction ;
28
29
import org .elasticsearch .action .support .IndicesOptions ;
30
+ import org .elasticsearch .client .Client ;
29
31
import org .elasticsearch .cluster .ClusterState ;
30
32
import org .elasticsearch .cluster .block .ClusterBlockLevel ;
31
33
import org .elasticsearch .cluster .metadata .IndexNameExpressionResolver ;
39
41
import org .elasticsearch .common .io .stream .Writeable ;
40
42
import org .elasticsearch .common .settings .Setting ;
41
43
import org .elasticsearch .common .settings .Setting .Property ;
44
+ import org .elasticsearch .common .util .concurrent .CountDown ;
42
45
import org .elasticsearch .index .Index ;
43
46
import org .elasticsearch .index .query .Rewriteable ;
44
47
import org .elasticsearch .index .shard .ShardId ;
50
53
import org .elasticsearch .threadpool .ThreadPool ;
51
54
import org .elasticsearch .transport .RemoteClusterAware ;
52
55
import org .elasticsearch .transport .RemoteClusterService ;
56
+ import org .elasticsearch .transport .RemoteTransportException ;
53
57
import org .elasticsearch .transport .Transport ;
54
58
import org .elasticsearch .transport .TransportService ;
55
59
60
64
import java .util .List ;
61
65
import java .util .Map ;
62
66
import java .util .Set ;
67
+ import java .util .concurrent .ConcurrentHashMap ;
63
68
import java .util .concurrent .Executor ;
64
69
import java .util .concurrent .TimeUnit ;
70
+ import java .util .concurrent .atomic .AtomicInteger ;
71
+ import java .util .concurrent .atomic .AtomicReference ;
65
72
import java .util .function .BiFunction ;
66
73
import java .util .function .Function ;
67
74
import java .util .function .LongSupplier ;
@@ -195,17 +202,23 @@ protected void doExecute(Task task, SearchRequest searchRequest, ActionListener<
195
202
executeSearch ((SearchTask )task , timeProvider , searchRequest , localIndices , Collections .emptyList (),
196
203
(clusterName , nodeId ) -> null , clusterState , Collections .emptyMap (), listener , SearchResponse .Clusters .EMPTY );
197
204
} else {
198
- remoteClusterService .collectSearchShards (searchRequest .indicesOptions (), searchRequest .preference (),
199
- searchRequest .routing (), remoteClusterIndices , ActionListener .wrap ((searchShardsResponses ) -> {
200
- List <SearchShardIterator > remoteShardIterators = new ArrayList <>();
201
- Map <String , AliasFilter > remoteAliasFilters = new HashMap <>();
202
- BiFunction <String , String , DiscoveryNode > clusterNodeLookup = processRemoteShards (searchShardsResponses ,
203
- remoteClusterIndices , remoteShardIterators , remoteAliasFilters );
204
- SearchResponse .Clusters clusters = buildClusters (localIndices , remoteClusterIndices , searchShardsResponses );
205
- executeSearch ((SearchTask ) task , timeProvider , searchRequest , localIndices ,
206
- remoteShardIterators , clusterNodeLookup , clusterState , remoteAliasFilters , listener ,
207
- clusters );
208
- }, listener ::onFailure ));
205
+ AtomicInteger skippedClusters = new AtomicInteger (0 );
206
+ collectSearchShards (searchRequest .indicesOptions (), searchRequest .preference (), searchRequest .routing (), skippedClusters ,
207
+ remoteClusterIndices , remoteClusterService , threadPool ,
208
+ ActionListener .wrap (
209
+ searchShardsResponses -> {
210
+ List <SearchShardIterator > remoteShardIterators = new ArrayList <>();
211
+ Map <String , AliasFilter > remoteAliasFilters = new HashMap <>();
212
+ BiFunction <String , String , DiscoveryNode > clusterNodeLookup = processRemoteShards (
213
+ searchShardsResponses , remoteClusterIndices , remoteShardIterators , remoteAliasFilters );
214
+ int localClusters = localIndices == null ? 0 : 1 ;
215
+ int totalClusters = remoteClusterIndices .size () + localClusters ;
216
+ int successfulClusters = searchShardsResponses .size () + localClusters ;
217
+ executeSearch ((SearchTask ) task , timeProvider , searchRequest , localIndices ,
218
+ remoteShardIterators , clusterNodeLookup , clusterState , remoteAliasFilters , listener ,
219
+ new SearchResponse .Clusters (totalClusters , successfulClusters , skippedClusters .get ()));
220
+ },
221
+ listener ::onFailure ));
209
222
}
210
223
}, listener ::onFailure );
211
224
if (searchRequest .source () == null ) {
@@ -216,18 +229,56 @@ protected void doExecute(Task task, SearchRequest searchRequest, ActionListener<
216
229
}
217
230
}
218
231
219
- static SearchResponse .Clusters buildClusters (OriginalIndices localIndices , Map <String , OriginalIndices > remoteIndices ,
220
- Map <String , ClusterSearchShardsResponse > searchShardsResponses ) {
221
- int localClusters = localIndices == null ? 0 : 1 ;
222
- int totalClusters = remoteIndices .size () + localClusters ;
223
- int successfulClusters = localClusters ;
224
- for (ClusterSearchShardsResponse searchShardsResponse : searchShardsResponses .values ()) {
225
- if (searchShardsResponse != ClusterSearchShardsResponse .EMPTY ) {
226
- successfulClusters ++;
227
- }
232
+ static void collectSearchShards (IndicesOptions indicesOptions , String preference , String routing , AtomicInteger skippedClusters ,
233
+ Map <String , OriginalIndices > remoteIndicesByCluster , RemoteClusterService remoteClusterService ,
234
+ ThreadPool threadPool , ActionListener <Map <String , ClusterSearchShardsResponse >> listener ) {
235
+ final CountDown responsesCountDown = new CountDown (remoteIndicesByCluster .size ());
236
+ final Map <String , ClusterSearchShardsResponse > searchShardsResponses = new ConcurrentHashMap <>();
237
+ final AtomicReference <RemoteTransportException > transportException = new AtomicReference <>();
238
+ for (Map .Entry <String , OriginalIndices > entry : remoteIndicesByCluster .entrySet ()) {
239
+ final String clusterAlias = entry .getKey ();
240
+ boolean skipUnavailable = remoteClusterService .isSkipUnavailable (clusterAlias );
241
+ Client clusterClient = remoteClusterService .getRemoteClusterClient (threadPool , clusterAlias );
242
+ final String [] indices = entry .getValue ().indices ();
243
+ ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest (indices )
244
+ .indicesOptions (indicesOptions ).local (true ).preference (preference ).routing (routing );
245
+ clusterClient .admin ().cluster ().searchShards (searchShardsRequest , new ActionListener <ClusterSearchShardsResponse >() {
246
+ @ Override
247
+ public void onResponse (ClusterSearchShardsResponse response ) {
248
+ searchShardsResponses .put (clusterAlias , response );
249
+ maybeFinish ();
250
+ }
251
+
252
+ @ Override
253
+ public void onFailure (Exception e ) {
254
+ if (skipUnavailable ) {
255
+ skippedClusters .incrementAndGet ();
256
+ } else {
257
+ RemoteTransportException exception =
258
+ new RemoteTransportException ("error while communicating with remote cluster [" + clusterAlias + "]" , e );
259
+ if (transportException .compareAndSet (null , exception ) == false ) {
260
+ transportException .accumulateAndGet (exception , (previous , current ) -> {
261
+ current .addSuppressed (previous );
262
+ return current ;
263
+ });
264
+ }
265
+ }
266
+ maybeFinish ();
267
+ }
268
+
269
+ private void maybeFinish () {
270
+ if (responsesCountDown .countDown ()) {
271
+ RemoteTransportException exception = transportException .get ();
272
+ if (exception == null ) {
273
+ listener .onResponse (searchShardsResponses );
274
+ } else {
275
+ listener .onFailure (transportException .get ());
276
+ }
277
+ }
278
+ }
279
+ }
280
+ );
228
281
}
229
- int skippedClusters = totalClusters - successfulClusters ;
230
- return new SearchResponse .Clusters (totalClusters , successfulClusters , skippedClusters );
231
282
}
232
283
233
284
static BiFunction <String , String , DiscoveryNode > processRemoteShards (Map <String , ClusterSearchShardsResponse > searchShardsResponses ,
0 commit comments