-
Couldn't load subscription status.
- Fork 25.6k
Fix race condition in RemoteClusterService.collectNodes()
#131937
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
c0e0bf7
2a56cee
6377745
87ebe39
111720d
1433889
494c0b9
48f583a
d6c05d7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,5 @@ | ||
| pr: 131937 | ||
| summary: Fix race condition in `RemoteClusterService.collectNodes()` | ||
| area: Distributed | ||
| type: bug | ||
| issues: [] |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -11,6 +11,7 @@ | |
| import org.apache.logging.log4j.Level; | ||
| import org.elasticsearch.TransportVersion; | ||
| import org.elasticsearch.action.ActionListener; | ||
| import org.elasticsearch.action.LatchedActionListener; | ||
| import org.elasticsearch.action.OriginalIndices; | ||
| import org.elasticsearch.action.support.ActionTestUtils; | ||
| import org.elasticsearch.action.support.IndicesOptions; | ||
|
|
@@ -1060,6 +1061,79 @@ public void onFailure(Exception e) { | |
| } | ||
| } | ||
|
|
||
| public void testCollectNodesConcurrentWithSettingsChanges() throws IOException { | ||
| final List<DiscoveryNode> knownNodes_c1 = new CopyOnWriteArrayList<>(); | ||
|
|
||
| try ( | ||
| var c1N1 = startTransport( | ||
| "cluster_1_node_1", | ||
| knownNodes_c1, | ||
| VersionInformation.CURRENT, | ||
| TransportVersion.current(), | ||
| Settings.EMPTY | ||
| ); | ||
| var transportService = MockTransportService.createNewService( | ||
| Settings.EMPTY, | ||
| VersionInformation.CURRENT, | ||
| TransportVersion.current(), | ||
| threadPool, | ||
| null | ||
| ) | ||
| ) { | ||
| final var c1N1Node = c1N1.getLocalNode(); | ||
| knownNodes_c1.add(c1N1Node); | ||
| final var seedList = List.of(c1N1Node.getAddress().toString()); | ||
| transportService.start(); | ||
| transportService.acceptIncomingRequests(); | ||
|
|
||
| try (RemoteClusterService service = new RemoteClusterService(createSettings("cluster_1", seedList), transportService)) { | ||
| service.initializeRemoteClusters(); | ||
| assertTrue(service.isCrossClusterSearchEnabled()); | ||
| final var numTasks = between(3, 5); | ||
| final var taskLatch = new CountDownLatch(numTasks); | ||
|
|
||
| ESTestCase.startInParallel(numTasks, threadNumber -> { | ||
| if (threadNumber == 0) { | ||
| taskLatch.countDown(); | ||
| boolean isLinked = true; | ||
| while (taskLatch.getCount() != 0) { | ||
| final var latch = new CountDownLatch(1); | ||
| service.updateRemoteCluster( | ||
| "cluster_1", | ||
| createSettings("cluster_1", isLinked ? Collections.emptyList() : seedList), | ||
| new LatchedActionListener<>(ActionListener.noop(), latch) | ||
| ); | ||
| safeAwait(latch); | ||
JeremyDahlgren marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| isLinked = isLinked == false; | ||
| } | ||
| return; | ||
| } | ||
|
|
||
| // Verify collectNodes() always invokes the listener, even if the node is concurrently being unlinked. | ||
| try { | ||
| for (int i = 0; i < 1000; ++i) { | ||
| final var latch = new CountDownLatch(1); | ||
| service.collectNodes(Set.of("cluster_1"), new LatchedActionListener<>(new ActionListener<>() { | ||
| @Override | ||
| public void onResponse(BiFunction<String, String, DiscoveryNode> func) { | ||
| assertEquals(c1N1Node, func.apply("cluster_1", c1N1Node.getId())); | ||
| } | ||
|
|
||
| @Override | ||
| public void onFailure(Exception e) { | ||
| assertNotNull(e); | ||
|
||
| } | ||
| }, latch)); | ||
| safeAwait(latch); | ||
| } | ||
| } finally { | ||
| taskLatch.countDown(); | ||
| } | ||
| }); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public void testRemoteClusterSkipIfDisconnectedSetting() { | ||
| { | ||
| Settings settings = Settings.builder() | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not a fan of using
fastForwardlike this - it means the listener completes early, potentially while othercollectNodescalls are still running, which can cause duplicated work and other confusions. The same concern applies to thefastForwardcall that already existed.Instead, could we adjust this all to use a
RefCountingListener? That would (a) delay the completion properly and (b) collect multiple failures (up to 10) as well as (c) being more idiomatic of modern Elasticsearch.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks David, much cleaner!