|
13 | 13 | import org.elasticsearch.action.ActionListener; |
14 | 14 | import org.elasticsearch.action.ActionRequest; |
15 | 15 | import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction; |
| 16 | +import org.elasticsearch.action.admin.cluster.node.usage.TransportNodeUsageStatsForThreadPoolsAction; |
16 | 17 | import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; |
17 | 18 | import org.elasticsearch.action.support.ActionFilter; |
18 | 19 | import org.elasticsearch.action.support.ActionFilters; |
|
21 | 22 | import org.elasticsearch.cluster.routing.IndexShardRoutingTable; |
22 | 23 | import org.elasticsearch.cluster.routing.RoutingTable; |
23 | 24 | import org.elasticsearch.cluster.routing.ShardRouting; |
| 25 | +import org.elasticsearch.cluster.routing.allocation.WriteLoadConstraintSettings; |
24 | 26 | import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; |
25 | 27 | import org.elasticsearch.cluster.service.ClusterService; |
26 | 28 | import org.elasticsearch.common.Strings; |
|
39 | 41 | import org.elasticsearch.test.ESIntegTestCase; |
40 | 42 | import org.elasticsearch.test.InternalTestCluster; |
41 | 43 | import org.elasticsearch.test.transport.MockTransportService; |
| 44 | +import org.elasticsearch.threadpool.ThreadPool; |
42 | 45 | import org.elasticsearch.transport.TransportService; |
43 | 46 | import org.hamcrest.Matchers; |
44 | 47 |
|
45 | 48 | import java.util.ArrayList; |
46 | 49 | import java.util.Arrays; |
47 | 50 | import java.util.Collection; |
| 51 | +import java.util.Collections; |
48 | 52 | import java.util.List; |
49 | 53 | import java.util.Locale; |
50 | 54 | import java.util.Map; |
51 | 55 | import java.util.Set; |
| 56 | +import java.util.concurrent.CountDownLatch; |
52 | 57 | import java.util.concurrent.atomic.AtomicBoolean; |
53 | 58 |
|
54 | 59 | import static java.util.Collections.emptySet; |
55 | 60 | import static java.util.Collections.singletonList; |
56 | 61 | import static java.util.Collections.unmodifiableSet; |
| 62 | +import static org.elasticsearch.cluster.InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING; |
57 | 63 | import static org.elasticsearch.common.util.set.Sets.newHashSet; |
58 | 64 | import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; |
59 | 65 | import static org.hamcrest.CoreMatchers.equalTo; |
@@ -202,7 +208,7 @@ public void testClusterInfoServiceInformationClearOnError() { |
202 | 208 | internalCluster().startNodes( |
203 | 209 | 2, |
204 | 210 | // manually control publishing |
205 | | - Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.getKey(), "60m").build() |
| 211 | + Settings.builder().put(INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.getKey(), "60m").build() |
206 | 212 | ); |
207 | 213 | prepareCreate("test").setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)).get(); |
208 | 214 | ensureGreen("test"); |
@@ -334,4 +340,65 @@ public void testClusterInfoServiceInformationClearOnError() { |
334 | 340 | ); |
335 | 341 | } |
336 | 342 | } |
| 343 | + |
| 344 | + public void testClusterInfoIncludesNodeUsageStatsForThreadPools() { |
| 345 | + var settings = Settings.builder() |
| 346 | + .put( |
| 347 | + WriteLoadConstraintSettings.WRITE_LOAD_DECIDER_ENABLED_SETTING.getKey(), |
| 348 | + WriteLoadConstraintSettings.WriteLoadDeciderStatus.ENABLED |
| 349 | + ) |
| 350 | + .build(); |
| 351 | + var masterName = internalCluster().startMasterOnlyNode(settings); |
| 352 | + var dataNodeName = internalCluster().startDataOnlyNode(settings); |
| 353 | + ensureStableCluster(2); |
| 354 | + assertEquals(internalCluster().getMasterName(), masterName); |
| 355 | + assertNotEquals(internalCluster().getMasterName(), dataNodeName); |
| 356 | + logger.info("---> master node: " + masterName + ", data node: " + dataNodeName); |
| 357 | + |
| 358 | + // Track when the data node receives a poll from the master for the write thread pool's stats. |
| 359 | + final MockTransportService dataNodeMockTransportService = MockTransportService.getInstance(dataNodeName); |
| 360 | + final CountDownLatch nodeThreadPoolStatsPolledByMaster = new CountDownLatch(1); |
| 361 | + dataNodeMockTransportService.addRequestHandlingBehavior( |
| 362 | + TransportNodeUsageStatsForThreadPoolsAction.NAME + "[n]", |
| 363 | + (handler, request, channel, task) -> { |
| 364 | + handler.messageReceived(request, channel, task); |
| 365 | + |
| 366 | + if (nodeThreadPoolStatsPolledByMaster.getCount() > 0) { |
| 367 | + logger.info("---> Data node received a request for thread pool stats"); |
| 368 | + } |
| 369 | + nodeThreadPoolStatsPolledByMaster.countDown(); |
| 370 | + } |
| 371 | + ); |
| 372 | + |
| 373 | + // Do some writes to create some write thread pool activity. |
| 374 | + final String indexName = randomIdentifier(); |
| 375 | + for (int i = 0; i < randomIntBetween(1, 1000); i++) { |
| 376 | + index(indexName, Integer.toString(i), Collections.singletonMap("foo", "bar")); |
| 377 | + } |
| 378 | + |
| 379 | + // Force a refresh of the ClusterInfo state to collect fresh info from the data nodes. |
| 380 | + final InternalClusterInfoService masterClusterInfoService = asInstanceOf( |
| 381 | + InternalClusterInfoService.class, |
| 382 | + internalCluster().getCurrentMasterNodeInstance(ClusterInfoService.class) |
| 383 | + ); |
| 384 | + final ClusterInfo clusterInfo = ClusterInfoServiceUtils.refresh(masterClusterInfoService); |
| 385 | + |
| 386 | + // Verify that the data node received a request for thread pool stats. |
| 387 | + safeAwait(nodeThreadPoolStatsPolledByMaster); |
| 388 | + |
| 389 | + final Map<String, NodeUsageStatsForThreadPools> usageStatsForThreadPools = clusterInfo.getNodeUsageStatsForThreadPools(); |
| 390 | + logger.info("---> Thread pool usage stats reported by data nodes to the master: " + usageStatsForThreadPools); |
| 391 | + assertThat(usageStatsForThreadPools.size(), equalTo(2)); // master and data node |
| 392 | + var dataNodeId = getNodeId(dataNodeName); |
| 393 | + var nodeUsageStatsForThreadPool = usageStatsForThreadPools.get(dataNodeId); |
| 394 | + assertNotNull(nodeUsageStatsForThreadPool); |
| 395 | + logger.info("---> Data node's thread pool stats: " + nodeUsageStatsForThreadPool); |
| 396 | + |
| 397 | + assertEquals(dataNodeId, nodeUsageStatsForThreadPool.nodeId()); |
| 398 | + var writeThreadPoolStats = nodeUsageStatsForThreadPool.threadPoolUsageStatsMap().get(ThreadPool.Names.WRITE); |
| 399 | + assertNotNull("Expected to find stats for the WRITE thread pool", writeThreadPoolStats); |
| 400 | + assertThat(writeThreadPoolStats.totalThreadPoolThreads(), greaterThan(0)); |
| 401 | + assertThat(writeThreadPoolStats.averageThreadPoolUtilization(), greaterThan(0f)); |
| 402 | + assertThat(writeThreadPoolStats.maxThreadPoolQueueLatencyMillis(), greaterThanOrEqualTo(0L)); |
| 403 | + } |
337 | 404 | } |
0 commit comments