21
21
22
22
import com .google .common .collect .ImmutableMap ;
23
23
import org .elasticsearch .action .ActionListener ;
24
+ import org .elasticsearch .action .LatchedActionListener ;
24
25
import org .elasticsearch .action .admin .cluster .node .stats .NodeStats ;
25
26
import org .elasticsearch .action .admin .cluster .node .stats .NodesStatsRequest ;
26
27
import org .elasticsearch .action .admin .cluster .node .stats .NodesStatsResponse ;
42
43
import org .elasticsearch .node .settings .NodeSettingsService ;
43
44
import org .elasticsearch .threadpool .ThreadPool ;
44
45
45
- import java .util .HashMap ;
46
- import java .util .Map ;
46
+ import java .util .*;
47
+ import java .util .concurrent .CountDownLatch ;
48
+ import java .util .concurrent .TimeUnit ;
47
49
48
50
/**
49
51
* InternalClusterInfoService provides the ClusterInfoService interface,
56
58
* Every time the timer runs, gathers information about the disk usage and
57
59
* shard sizes across the cluster.
58
60
*/
59
- public final class InternalClusterInfoService extends AbstractComponent implements ClusterInfoService , LocalNodeMasterListener , ClusterStateListener {
61
+ public class InternalClusterInfoService extends AbstractComponent implements ClusterInfoService , LocalNodeMasterListener , ClusterStateListener {
60
62
61
63
public static final String INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL = "cluster.info.update.interval" ;
62
64
@@ -70,6 +72,7 @@ public final class InternalClusterInfoService extends AbstractComponent implemen
70
72
private final TransportIndicesStatsAction transportIndicesStatsAction ;
71
73
private final ClusterService clusterService ;
72
74
private final ThreadPool threadPool ;
75
+ private final Set <Listener > listeners = Collections .synchronizedSet (new HashSet <Listener >());
73
76
74
77
@ Inject
75
78
public InternalClusterInfoService (Settings settings , NodeSettingsService nodeSettingsService ,
@@ -188,6 +191,11 @@ public ClusterInfo getClusterInfo() {
188
191
return new ClusterInfo (usages , shardSizes );
189
192
}
190
193
194
+ @ Override
195
+ public void addListener (Listener listener ) {
196
+ this .listeners .add (listener );
197
+ }
198
+
191
199
/**
192
200
* Class used to submit {@link ClusterInfoUpdateJob}s on the
193
201
* {@link InternalClusterInfoService} threadpool, these jobs will
@@ -210,6 +218,34 @@ public void run() {
210
218
}
211
219
}
212
220
221
+ /**
222
+ * Retrieve the latest nodes stats, calling the listener when complete
223
+ * @return a latch that can be used to wait for the nodes stats to complete if desired
224
+ */
225
+ protected CountDownLatch updateNodeStats (final ActionListener <NodesStatsResponse > listener ) {
226
+ final CountDownLatch latch = new CountDownLatch (1 );
227
+ final NodesStatsRequest nodesStatsRequest = new NodesStatsRequest ("data:true" );
228
+ nodesStatsRequest .clear ();
229
+ nodesStatsRequest .fs (true );
230
+ nodesStatsRequest .timeout (TimeValue .timeValueSeconds (15 ));
231
+
232
+ transportNodesStatsAction .execute (nodesStatsRequest , new LatchedActionListener <>(listener , latch ));
233
+ return latch ;
234
+ }
235
+
236
+ /**
237
+ * Retrieve the latest indices stats, calling the listener when complete
238
+ * @return a latch that can be used to wait for the indices stats to complete if desired
239
+ */
240
+ protected CountDownLatch updateIndicesStats (final ActionListener <IndicesStatsResponse > listener ) {
241
+ final CountDownLatch latch = new CountDownLatch (1 );
242
+ final IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest ();
243
+ indicesStatsRequest .clear ();
244
+ indicesStatsRequest .store (true );
245
+
246
+ transportIndicesStatsAction .execute (indicesStatsRequest , new LatchedActionListener <>(listener , latch ));
247
+ return latch ;
248
+ }
213
249
214
250
/**
215
251
* Runnable class that performs a {@Link NodesStatsRequest} to retrieve
@@ -252,12 +288,7 @@ public void run() {
252
288
return ;
253
289
}
254
290
255
- NodesStatsRequest nodesStatsRequest = new NodesStatsRequest ("data:true" );
256
- nodesStatsRequest .clear ();
257
- nodesStatsRequest .fs (true );
258
- nodesStatsRequest .timeout (TimeValue .timeValueSeconds (15 ));
259
-
260
- transportNodesStatsAction .execute (nodesStatsRequest , new ActionListener <NodesStatsResponse >() {
291
+ CountDownLatch nodeLatch = updateNodeStats (new ActionListener <NodesStatsResponse >() {
261
292
@ Override
262
293
public void onResponse (NodesStatsResponse nodeStatses ) {
263
294
Map <String , DiskUsage > newUsages = new HashMap <>();
@@ -294,10 +325,7 @@ public void onFailure(Throwable e) {
294
325
}
295
326
});
296
327
297
- IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest ();
298
- indicesStatsRequest .clear ();
299
- indicesStatsRequest .store (true );
300
- transportIndicesStatsAction .execute (indicesStatsRequest , new ActionListener <IndicesStatsResponse >() {
328
+ CountDownLatch indicesLatch = updateIndicesStats (new ActionListener <IndicesStatsResponse >() {
301
329
@ Override
302
330
public void onResponse (IndicesStatsResponse indicesStatsResponse ) {
303
331
ShardStats [] stats = indicesStatsResponse .getShards ();
@@ -325,8 +353,24 @@ public void onFailure(Throwable e) {
325
353
}
326
354
});
327
355
328
- if (logger .isTraceEnabled ()) {
329
- logger .trace ("Finished ClusterInfoUpdateJob" );
356
+ try {
357
+ nodeLatch .await (15 , TimeUnit .SECONDS );
358
+ } catch (InterruptedException e ) {
359
+ logger .warn ("Failed to update node information for ClusterInfoUpdateJob within 15s timeout" );
360
+ }
361
+
362
+ try {
363
+ indicesLatch .await (15 , TimeUnit .SECONDS );
364
+ } catch (InterruptedException e ) {
365
+ logger .warn ("Failed to update shard information for ClusterInfoUpdateJob within 15s timeout" );
366
+ }
367
+
368
+ for (Listener l : listeners ) {
369
+ try {
370
+ l .onNewInfo (getClusterInfo ());
371
+ } catch (Exception e ) {
372
+ logger .info ("Failed executing ClusterInfoService listener" , e );
373
+ }
330
374
}
331
375
}
332
376
}
0 commit comments