georgew5656 opened a new pull request, #17535:
URL: https://github.com/apache/druid/pull/17535
Sometimes the LifecycleStop method of SupervisorManager
(SupervisorManager.stop()) can take a long time to run. This is because the
method iterates through all running supervisors and calls stop on them
serially. Each streaming supervisor.stop() call tries to push a ShutdownNotice
to its notice queue and then wait for the ShutdownNotice to run and set stopped
= true up to tuningConfig.shutdownTimeout. This means the total run time can be
the sum of tuningConfig.shutdownTimeout (default 80 seconds) across all
supervisors.
This long stop time can cause lots of issues, most notably overlord
leadership issues if the ZK leader is terminated (but the ensemble maintains
quorum). This is because a overlord pod can get becomeLeader queued up behind
stopLeader if it disconnects and then reconnects to ZK (the giant lock shared
between the two methods).
This PR attempts to ensure SupervisorManager completes faster to prevent
this issue. (although I feel some of the leadership process on the overlord
specifically maybe needs to be revisited in general).
Still working on some unit tests for this change
### Description
- In SupervisorManager use a static pool of shutdownThreads to stop
supervisors in parallel in the stop method to prevent a single or few slow
supervisors from slowing down overall shutdown.
- In SeekableStreamSupervisor, when stopGracefully is false (as it is when
we are shutting down SupervisorManager), don't wait for the ShutdownNotice to
run. This means that the recordSupplier (e.g. kafka consumer) may not be
cleaned up immediately, but since all the supervisor objects are dereferenced
and can be GC'd later i don't think this is a huge deal.
#### Fixed the bug ...
#### Renamed the class ...
#### Added a forbidden-apis entry ...
- I used a static thread pool in SupervisorManager for now, it's possible it
should be configurable but IMO the main point of the thread pool is to not let
a few slow supervisor shutdowns run in series block the entire supervisor
manager from shutting down.
- I'm not sure if changing the SeekableStreamSupervisor.stop method when
stopGracefully = false is necessary, but it didn't make sense to me to specify
a non-graceful shutdown and then try to wait for things to clean up.
#### Release note
Improve recovery time for overlord leadership after zk nodes are bounced.
##### Key changed/added classes in this PR
* `SupervisorManager`
* `SeekableStreamSupervisor`
This PR has:
- [X] been self-reviewed.
- [ ] using the [concurrency
checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md)
(Remove this item if the PR doesn't have any relation to concurrency.)
- [ ] added documentation for new or modified features or behaviors.
- [ ] a release note entry in the PR description.
- [ ] added Javadocs for most classes and all non-trivial methods. Linked
related entities via Javadoc links.
- [ ] added or updated version, license, or notice information in
[licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
- [ ] added comments explaining the "why" and the intent of the code
wherever would not be obvious for an unfamiliar reader.
- [ ] added unit tests or modified existing tests to cover new code paths,
ensuring the threshold for [code
coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md)
is met.
- [ ] added integration tests.
- [X] been tested in a test Druid cluster.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]