|
42 | 42 | import org.elasticsearch.common.transport.TransportAddress;
|
43 | 43 | import org.elasticsearch.common.unit.TimeValue;
|
44 | 44 | import org.elasticsearch.common.util.CancellableThreads;
|
| 45 | +import org.elasticsearch.common.util.concurrent.ThreadContext; |
45 | 46 | import org.elasticsearch.common.xcontent.XContentBuilder;
|
46 | 47 | import org.elasticsearch.common.xcontent.XContentFactory;
|
47 | 48 | import org.elasticsearch.core.internal.io.IOUtils;
|
@@ -555,6 +556,64 @@ public void testFetchShards() throws Exception {
|
555 | 556 | }
|
556 | 557 | }
|
557 | 558 |
|
| 559 | + public void testFetchShardsThreadContextHeader() throws Exception { |
| 560 | + List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>(); |
| 561 | + try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT); |
| 562 | + MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) { |
| 563 | + DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); |
| 564 | + knownNodes.add(seedTransport.getLocalDiscoNode()); |
| 565 | + knownNodes.add(discoverableTransport.getLocalDiscoNode()); |
| 566 | + Collections.shuffle(knownNodes, random()); |
| 567 | + try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { |
| 568 | + service.start(); |
| 569 | + service.acceptIncomingRequests(); |
| 570 | + List<DiscoveryNode> nodes = Collections.singletonList(seedNode); |
| 571 | + try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", |
| 572 | + nodes, service, Integer.MAX_VALUE, n -> true)) { |
| 573 | + SearchRequest request = new SearchRequest("test-index"); |
| 574 | + Thread[] threads = new Thread[10]; |
| 575 | + for (int i = 0; i < threads.length; i++) { |
| 576 | + final String threadId = Integer.toString(i); |
| 577 | + threads[i] = new Thread(() -> { |
| 578 | + ThreadContext threadContext = seedTransport.threadPool.getThreadContext(); |
| 579 | + threadContext.putHeader("threadId", threadId); |
| 580 | + AtomicReference<ClusterSearchShardsResponse> reference = new AtomicReference<>(); |
| 581 | + AtomicReference<Exception> failReference = new AtomicReference<>(); |
| 582 | + final ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest("test-index") |
| 583 | + .indicesOptions(request.indicesOptions()).local(true).preference(request.preference()) |
| 584 | + .routing(request.routing()); |
| 585 | + CountDownLatch responseLatch = new CountDownLatch(1); |
| 586 | + connection.fetchSearchShards(searchShardsRequest, |
| 587 | + new LatchedActionListener<>(ActionListener.wrap( |
| 588 | + resp -> { |
| 589 | + reference.set(resp); |
| 590 | + assertEquals(threadId, seedTransport.threadPool.getThreadContext().getHeader("threadId")); |
| 591 | + }, |
| 592 | + failReference::set), responseLatch)); |
| 593 | + try { |
| 594 | + responseLatch.await(); |
| 595 | + } catch (InterruptedException e) { |
| 596 | + throw new RuntimeException(e); |
| 597 | + } |
| 598 | + assertNull(failReference.get()); |
| 599 | + assertNotNull(reference.get()); |
| 600 | + ClusterSearchShardsResponse clusterSearchShardsResponse = reference.get(); |
| 601 | + assertEquals(knownNodes, Arrays.asList(clusterSearchShardsResponse.getNodes())); |
| 602 | + }); |
| 603 | + } |
| 604 | + for (int i = 0; i < threads.length; i++) { |
| 605 | + threads[i].start(); |
| 606 | + } |
| 607 | + |
| 608 | + for (int i = 0; i < threads.length; i++) { |
| 609 | + threads[i].join(); |
| 610 | + } |
| 611 | + assertTrue(connection.assertNoRunningConnections()); |
| 612 | + } |
| 613 | + } |
| 614 | + } |
| 615 | + } |
| 616 | + |
558 | 617 | public void testFetchShardsSkipUnavailable() throws Exception {
|
559 | 618 | List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
|
560 | 619 | try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT)) {
|
|
0 commit comments