Skip to content
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

Nonblocking master service #94325

Conversation

DaveCTurner
Copy link
Contributor

The changes introduced in #92021 mean that there is no need for the master service to block its thread while waiting for each publication to complete. This commit removes the now-unnecessary blocking.

This commit also removes the now-unnecessary fake blocking in the FakeThreadPoolMasterService used in tests, bringing the implementation covered by tests closer to the production implementation.

@DaveCTurner DaveCTurner added >non-issue :Distributed Coordination/Cluster Coordination Cluster formation and cluster state publication, including cluster membership and fault detection. v8.8.0 labels Mar 6, 2023
The changes introduced in elastic#92021 mean that there is no need for the
master service to block its thread while waiting for each publication to
complete. This commit removes the now-unnecessary blocking.

This commit also removes the now-unnecessary fake blocking in the
`FakeThreadPoolMasterService` used in tests, bringing the implementation
covered by tests closer to the production implementation.
@DaveCTurner DaveCTurner force-pushed the 2023-03-06-nonblocking-master-service branch from 844ad98 to b0a4aa8 Compare March 6, 2023 13:36
@@ -2369,7 +2377,7 @@ public void taskSucceeded(ClusterStateUpdateTask clusterStateTaskListener, Void
public ClusterState execute(ClusterState currentState) {
assertEquals(priority, prioritiesQueue.poll());
assertEquals(priority, priority());
return currentState;
return randomBoolean() ? currentState : ClusterState.builder(currentState).build();
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We're using a single-threaded executor here, so by sometimes triggering a publication and sometimes forking the publication handling onto a separate thread we demonstrate that there is no longer any blocking involved in the publication process.

@DaveCTurner DaveCTurner marked this pull request as ready for review March 6, 2023 15:23
@elasticsearchmachine elasticsearchmachine added the Team:Distributed (Obsolete) Meta label for distributed team (obsolete). Replaced by Distributed Indexing/Coordination. label Mar 6, 2023
@elasticsearchmachine
Copy link
Collaborator

Pinging @elastic/es-distributed (Team:Distributed)

Copy link
Contributor Author

@DaveCTurner DaveCTurner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

tip: ignoring whitespace will make this easier to review

@fcofdez fcofdez self-requested a review March 6, 2023 17:54
Copy link
Contributor

@henningandersen henningandersen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this looks good, but have a question/comment that you can maybe clarify quickly?

@DaveCTurner
Copy link
Contributor Author

Bah, ok, we're not always on the master thread where I thought we were (because rejections might leave us on the applier thread). I'll remove that comment for now and follow up later.

Copy link
Contributor

@henningandersen henningandersen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

threadPool.executor(randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC)).execute(() -> {
final var threadName = Thread.currentThread().getName();
try {
Thread.currentThread().setName("[" + MasterService.MASTER_UPDATE_THREAD_NAME + "]");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems unnecessary now that we use the ThreadedActionListener. I think we should remove this line (and the try-finally), that would validate that we can call the publish listener from any thread.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, yes - see 464358c.

@DaveCTurner DaveCTurner added the auto-merge-without-approval Automatically merge pull request when CI checks pass (NB doesn't wait for reviews!) label Mar 30, 2023
@elasticsearchmachine elasticsearchmachine merged commit 356e109 into elastic:main Mar 30, 2023
@DaveCTurner DaveCTurner deleted the 2023-03-06-nonblocking-master-service branch March 30, 2023 10:56
DaveCTurner added a commit to DaveCTurner/elasticsearch that referenced this pull request Mar 30, 2023
These tests submit tasks to the master service and wait for the state to
be applied, but not for the publication to complete, and without that
wait they may clean up too soon. This commit makes them wait for the
master service to finish its work before cleaning up.

Relates elastic#94325
Closes elastic#94900
elasticsearchmachine pushed a commit that referenced this pull request Mar 30, 2023
These tests submit tasks to the master service and wait for the state to
be applied, but not for the publication to complete, and without that
wait they may clean up too soon. This commit makes them wait for the
master service to finish its work before cleaning up.

Relates #94325 Closes #94900
DaveCTurner added a commit to DaveCTurner/elasticsearch that referenced this pull request Apr 5, 2023
In elastic#94325 we introduced another forking step when submitting a
publication, so we must extend the timeout in this test (and
`DEFAULT_CLUSTER_STATE_UPDATE_DELAY`) by `DEFAULT_DELAY_VARIABILITY`.

Closes elastic#94905
elasticsearchmachine pushed a commit that referenced this pull request Apr 5, 2023
In #94325 we introduced another forking step when submitting a
publication, so we must extend the timeout in this test (and
`DEFAULT_CLUSTER_STATE_UPDATE_DELAY`) by `DEFAULT_DELAY_VARIABILITY`.

Closes #94905
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
auto-merge-without-approval Automatically merge pull request when CI checks pass (NB doesn't wait for reviews!) :Distributed Coordination/Cluster Coordination Cluster formation and cluster state publication, including cluster membership and fault detection. >non-issue Team:Distributed (Obsolete) Meta label for distributed team (obsolete). Replaced by Distributed Indexing/Coordination. v8.8.0
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants