Core, AWS: Allow stopping thread pools manually#15312
Core, AWS: Allow stopping thread pools manually#15312svalaskevicius wants to merge 33 commits intoapache:mainfrom
Conversation
7f955d4 to
cbb17a3
Compare
- allows to stop thread pools manually, to avoid leaks in hot-reload environments - allows to opt-out of the standard shutdown mechanism to manage graceful service stops (and commit the last pending files)
cbb17a3 to
db22fba
Compare
I've updated the javadoc to word it more seriously - to only call this at the end of the intended usage of the library, and never before. Also, it is safe to call it multiple times -- but only AFTER the client code is sure that it's not longer using the library |
mxm
left a comment
There was a problem hiding this comment.
A couple of more comments:
- Why are we not touching AuthRefreshPoolHolder?
- There are no tests yet.
|
@mxm updated, please check. I had missed also, what tests do you have in mind? this is still all pretty much static code and invoking the shutdown in test affects a lot. I suppose one option would be to extract functions for the more complicated, non-obvious features, and test them in isolation - do you have any preferences? I'm a bit wary of the nondeterministic state of this PR/task - esp with the new parallel PR out there :) would it be possible to have a think and set out a single list of changes pending before it can be considered complete/good enough? Thanks! |
186f578 to
ff4d7e0
Compare
This reverts commit f6c1d52.
|
@svalaskevicius can you please update the description with exactly what this is changing and why? I think it is a red flag that this is touching the static worker pool, but doesn't specifically call that out in the PR description and also doesn't explain why that is needed. If this PR is about how to manage threadpools coming from factory methods in I also am skeptical of this kind of update in general. This class was intended to hold a single threadpool, not to provide threadpools. As it grew over time, people wanted to have threadpools configured the same way. But changing the behavior of the default pools in order to make other uses of the convenience methods more generic is not the right path forward. |
rdblue
left a comment
There was a problem hiding this comment.
I think that this should not touch the default threadpools and should clearly outline the motivation for these changes in the PR description.
If I understand correctly from #15031, it looks like the issue is that Flink uses new classloaders and the shutdown hook is never called because the JVM never exits. So we need use threadpools tied to Flink's lifecycle. That's reasonable, but we definitely do NOT want to give "users" control over threadpool lifecycles. I think this implementation goes a bit beyond what we need to do. Assuming that we want to have a way to shut down these pools, that doesn't mean that we should remove the shutdown hook. The contract of Also, it's debatable whether we need to track all pools created by the factory methods here. This class is responsible for its own static pools and its methods are reused elsewhere. Having a way to close all threadpools created by this class seems very risky to me. I'm not convinced that we want to expose a method to shut down a single static threadpool directly, let alone one that will shut down pools created and managed by other classes. The reason for static pools is to ensure one is available. Allowing them to be closed allows one class to break functionality for another. Allowing all of them to be closed is even more risk. I think the solution is not to expose these methods. Instead, I propose two things:
I think if we do these two things then we will fix the problem and be better off. I'd also be okay with an option to prevent these pools from being started if we need more guarantees. I think that's safer and better than using them but shutting them down. |
Technically, nothing prevents users from shutting down the static pools already, for example: Today we mostly rely on reviews to prevent this, and in some cases it is not necessarily obvious. If the goal is to guarantee that users cannot shut these pools down, that could be addressed more directly (e.g., by wrapping the returned executor), but that safeguard does not exist today. Changing all usages of Do you think a better long‑term direction would be to change and propagate this through the In practice, and propagate this through the The FileIO implementations can't manage the lifecycle for the pools as they don't know if they should keep the shared resources open or they should close them too. |
|
NOTE: updated the description again to add a section about the non-deterministic nature of the JVM shutdown hooks and how it prevents clean iceberg file commits when the application is exiting |
|
@pvary, let me outline my assumptions so that my rationale is clear. First, I'm assuming that we are only worrying about the exiting pools. The other pools should have their own lifecycle management. The non-exiting pools are what I would expect However, I don't think the
I think those are both poor solutions. For the first one, if you don't want a threadpool that is configured like the static pools, the right answer is to use your own threadpool and manage its lifecycle. Customizing the lifecycle of the static pools opens the possibility of misconfiguration. The reasonable way to allow using your own threadpool (and not worrying about the shutdown hooks) is to lazily start the static pools. But if you can already shut down the static pools, then we don't really need even that. For the second one, I think it is a bad idea to have a global shutdown for the static pools, let along all of the pools created by the factory methods. This is a drastic behavior change that can be misused. It may allow us to avoid fixing pools created by |
|
Hi @rdblue, Just to clarify:
The problem covers all thread pools currently managed by MoreExecutors - including both worker pools here.
A global shutdown exists already via MoreExecutors, just that it is deferred to the JVM shutdown hook and there is no control as to when exactly will it be invoked.
I agree with this. If iceberg had thread pools injected via constructors (and without relying on the global JVM shutdown hooks) it would address the issue cleanly. At this point, however, I was looking for a pragmatic solution. We have an application that uses the iceberg library, and cannot commit/flush files on exit because iceberg has already killed its threads and is producing a lot of errors how the ThreadPools have been stopped. Given that, and that there is an open issue related to thread stopping, my reasoning was that these can be combined to a single change to solve both problems - this is the resulting PR. I suppose the question is - is this an acceptable interim solution until the threadpool (factory?) injection is implemented, or should this PR be closed and converted to an open issue (for the uncontrolled/non-deterministic shutdown behaviour)? Do you have any thoughts as to when could the proper fix be expected? Regards, |
|
@rdblue: Thanks for the detailed reply. This helps clarify where our views differ.
Agreed.
Fully agree.
In practice, all
The naming may indeed need improvement, but the proposed shutdown only targets the exiting pools created by
From my perspective, the pattern across Based on this discussion, I see a few possible directions:
Independently of which path we take, I think we should ensure that standard pools are lazily initialized, and that every place using them (using the methods: I would prefer 2 or 3. Do you think it would be reasonable? Do you have another suggestion? |
Control of static thread pools — manual shutdown and lifecycle management
This change introduces a centralized
ThreadPoolManagerthat gives users explicit control over Iceberg's thread pool lifecycle.Motivation
The previous approach used Guava's
MoreExecutors.getExitingExecutorService, which registered unremovable shutdown hooks that accumulated over time with short-lived pools. This new design replaces it with a central manager, enabling manual shutdown and opt-out of automatic hook registration.This is needed when the application registers its own shutdown hooks, that need to commit the iceberg file upload cleanly. Because JVM does not allow control of the order in which the shutdown hooks get invoked, the problem with the current state is that iceberg kills its thread pools and is unable to complete the export.
Intended Use Cases
Key Changes
shutdownThreadPools()— gracefully shuts down all registered pools and removes the JVM shutdown hookremoveShutdownHook()— opt out of automatic shutdown hooks so applications can manage their own graceful shutdown sequence (within their shutdown hooks)Fixes issue #15039