Parallelize supervisor stop logic to make it run faster#17535
Parallelize supervisor stop logic to make it run faster#17535kfaraz merged 10 commits intoapache:masterfrom
Conversation
| } | ||
| log.info("Waiting for [%d] supervisors to shutdown", stopFutures.size()); | ||
| try { | ||
| FutureUtils.coalesce(stopFutures).get(80, TimeUnit.SECONDS); |
There was a problem hiding this comment.
I don't think we should use a timeout of 80s here since each supervisor could have a different value of shutdown timeout. We could either just do get() with no args (which would be no worse than what the code is currently doing) or use a longer timeout.
| { | ||
| this.metadataSupervisorManager = metadataSupervisorManager; | ||
| this.shutdownExec = MoreExecutors.listeningDecorator( | ||
| Execs.multiThreaded(25, "supervisor-manager-shutdown-%d") |
There was a problem hiding this comment.
25 may be excessive in some cases and inadequate in others. Maybe initialize the executor lazily inside the stop() method, then the number of required threads can be computed at run time. The shutdownExec need not be a class-level field either.
Alternatively, instead of using a completely new executor, you could consider using the scheduledExec inside each supervisor. That executor basically just sits idle most of the time and is responsible only for submitting RunNotice to the notice queue.
You could add a stopAsync method to SeekableStreamSupervisor that does the following:
- returns a future that we coalesce and wait upon
- internally submits a runnable to the
scheduledExecto perform the actualstop
I guess the only thing we will miss out on is parallelizing the autoscaler.stop() which should not be a concern, I guess?
There was a problem hiding this comment.
the issue i was running into with this strategy is that part of the stop logic is shutting down the scheduledExec executor, and I couldn't really think of a great way to avoid this chicken-and-egg problem.
There was a problem hiding this comment.
You could perhaps work around that problem by doing something like this:
stopAsyncsets the supervisor state to STOPPINGstopAsyncthen submits astop()runnable to thescheduledExecbuildRunTaskmethod should check and submit theRunNoticeonly if the state of the supervisor is not STOPPINGstop()can callscheduledExec.shutdown()instead ofscheduledExec.shutdownNow()
Another alternative is to simply create a shutdownExec inside stopAsync.
Difference from the current approach would be that the SupervisorManager doesn't need to handle the lifecycle of the shutdown executor.
Let me know what you think.
There was a problem hiding this comment.
i think this makes sense, will update
| if (!stopGracefully) { | ||
| stopped = true; | ||
| break; | ||
| } |
There was a problem hiding this comment.
If we have already parallelized the stop of supervisors, is this still needed?
There was a problem hiding this comment.
we could probably pull it out
| */ | ||
| void stop(boolean stopGracefully); | ||
|
|
||
| default ListenableFuture<Void> stopAsync(boolean stopGracefully) |
There was a problem hiding this comment.
Maybe add a javadoc.
Also, does this need the stopGracefully parameter?
| default ListenableFuture<Void> stopAsync(boolean stopGracefully) | ||
| { | ||
| SettableFuture<Void> stopFuture = SettableFuture.create(); | ||
| stop(stopGracefully); |
There was a problem hiding this comment.
If this method throws an exception, the future should be completed with the exception.
…blestream/supervisor/SeekableStreamSupervisor.java Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
|
Thanks for the changes, @georgew5656 ! |
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.
Description
Fixed the bug ...
Renamed the class ...
Added a forbidden-apis entry ...
Release note
Improve recovery time for overlord leadership after zk nodes are bounced.
Key changed/added classes in this PR
SupervisorManagerSeekableStreamSupervisorThis PR has: