Add OOM pause mechanism to pause query threads before killing on critical heap#18163
Add OOM pause mechanism to pause query threads before killing on critical heap#18163yashmayya wants to merge 4 commits intoapache:masterfrom
Conversation
Codecov Report❌ Patch coverage is Additional details and impacted files@@ Coverage Diff @@
## master #18163 +/- ##
============================================
+ Coverage 63.32% 63.36% +0.03%
Complexity 1627 1627
============================================
Files 3238 3238
Lines 197011 197089 +78
Branches 30466 30481 +15
============================================
+ Hits 124762 124877 +115
+ Misses 62244 62193 -51
- Partials 10005 10019 +14
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
xiangfu0
left a comment
There was a problem hiding this comment.
High-signal issue inline.
| threadTracker.updateMemorySnapshot(); | ||
| } | ||
| // Block if an OOM pause is active (fast-path: single volatile read) | ||
| _queryResourceAggregator.waitIfPaused(); |
There was a problem hiding this comment.
IMO this shouldn't be part of sample usage, but termination check: pause first, then check termination. Make sure termination is set before releasing pause
There was a problem hiding this comment.
That will require some SPI changes, how about we swap order of sampling usage and termination checks instead (i.e., sample first - pause if needed - check termination)?
There was a problem hiding this comment.
SPI change is fine. I'm thinking from an abstraction level, where there action belongs to the termination check, not usage sampling. E.g. the engine might only check termination without doing sampling
| if (remaining <= 0) { | ||
| break; | ||
| } | ||
| _pauseCondition.await(remaining, TimeUnit.MILLISECONDS); |
There was a problem hiding this comment.
Do we need timeout here, or only rely on condition variable?
There was a problem hiding this comment.
We could, but that relies solely on the watcher task thread to wake up query execution threads - this wait with timeout is like a double safeguard to ensure that query threads don't ever end up waiting indefinitely and appear stuck. Any particular issues with this wait with timeout approach?
There was a problem hiding this comment.
It is a little bit risky because thread might wake up before watcher thread tries to terminate the query, causing higher memory usage
| public static final String CONFIG_OF_OOM_PAUSE_ENABLED = "accounting.oom.critical.query.pause.enabled"; | ||
| public static final boolean DEFAULT_OOM_PAUSE_ENABLED = false; | ||
|
|
||
| public static final String CONFIG_OF_OOM_PAUSE_TIMEOUT_MS = "accounting.oom.critical.query.pause.timeout.ms"; |
There was a problem hiding this comment.
We can merge them into one config, where non-positive value (e.g. default -1) means no pause. Something like accounting.oom.query.kill.grace.period.ms
Should we also allow pausing when memory usage directly jumping into panic level? Or consider using a flag to control it.
There was a problem hiding this comment.
Yeah I can merge the configs into one, but I'm not sure query.kill.grace.period is the right terminology? It's not exactly a "grace period" we're offering the threads - which sounds like we're waiting for them to complete execution gracefully - we're actually pausing execution instead.
As for the point on panic, IMO that would significantly muddy the panic semantics. I'd argue panic should stay as immediate kill-all with no pause since the JVM is almost out of memory and any delay could tip into OOM.
There was a problem hiding this comment.
How about accounting.oom.pre.query.kill.pause.duration.ms? I feel pause.timeout is a little bit ambiguous.
I remember seeing memory usage directly jumping to panic sometimes. I agree it feels dangerous to not immediately kill on panic, but maybe it is good to at least have that option so we can try it out
There was a problem hiding this comment.
accounting.oom.pre.query.kill.pause.duration.ms sounds good. I've also added a new boolean config accounting.oom.panic.pre.query.kill.pause.enabled (defaulting to false) which can control the pause on direct transition to panic (bypassing critical).
ceb1cb6 to
677e123
Compare
| if (System.currentTimeMillis() >= _pauseDeadlineMs || !pauseOnPanic) { | ||
| // Grace period expired, or panic-pause disabled — kill all and clear | ||
| killAllQueries(threadTrackers); |
There was a problem hiding this comment.
Hm actually now that I think about it, we probably don't want to wait for grace period to expire on transition from critical to panic, even if the pause on panic config is enabled..
Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
| public static final long DEFAULT_OOM_PRE_QUERY_KILL_PAUSE_DURATION_MS = -1; | ||
|
|
||
| public static final String CONFIG_OF_OOM_PANIC_PRE_QUERY_KILL_PAUSE_ENABLED = | ||
| "accounting.oom.panic.pre.query.kill.pause.enabled"; |
There was a problem hiding this comment.
(optional) I feel it might be clearer:
| "accounting.oom.panic.pre.query.kill.pause.enabled"; | |
| "accounting.oom.panic.allow.pre.query.kill.pause"; |
| // NOTE: In production code, threadContext should never be null. It might be null in tests when QueryThreadContext | ||
| // is not set up. | ||
| if (threadContext != null) { | ||
| threadContext.waitIfPausedInternal(); |
There was a problem hiding this comment.
Discussed offline. Let's still check termination first, then when entering the pause, check termination again before exiting the pause. Pause should be done after the sampling
| /** | ||
| * Clears the OOM pause and wakes all blocked query threads. | ||
| */ | ||
| void clearPause() { |
There was a problem hiding this comment.
(minor) Is this exposed for testing purpose? Either annotate it or make it private. Same for other places
| private final ReentrantLock _pauseLock = new ReentrantLock(); | ||
| private final Condition _pauseCondition = _pauseLock.newCondition(); | ||
| private volatile boolean _pauseActive = false; | ||
| private volatile long _pauseDeadlineMs = 0; |
There was a problem hiding this comment.
(nit) Doesn't need to be volatile as it is maintained as a local state for the watcher.
Consider adding some comments describing the contract for them:
_pauseActive is shared by both watcher and query threads
_pauseDeadlineMs is local state for the watcher (similar to _triggeringLevel)
| } | ||
|
|
||
| private void evalTriggers() { | ||
| TriggeringLevel previousTriggeringLevel = _triggeringLevel; |
There was a problem hiding this comment.
(minor) This local variable is no longer needed
Summary
System.gc()(only once, on first transition to critical), giving the JVM breathing room to reclaim memory before resorting to query kills.Overhead
waitIfPaused()is called at every sampling checkpoint and costs a single volatile read of_pauseActive. No lock, no contention.ReentrantLock+Condition. Lock contention is not a concern because:Condition.await(), releasing it immediately - hold time is near-zero.activatePause()(watcher thread) is lock-free - it writes two volatiles with correct ordering.Configuration
Both are dynamically updatable via Helix cluster config.
State transitions
System.gc()