[SPARK-45025][CORE] Allow block manager memory store iterator to handle thread interrupt and perform task completion gracefully#42742
Conversation
…ead interrupt and perform task completion gracefully
|
cc - @JoshRosen , @HeartSaVioR - PTAL, thx ! |
|
cc. @jiangxb1987 @Ngone51 |
HeartSaVioR
left a comment
There was a problem hiding this comment.
The rationale and the fix seems to be OK, but I doubt I'm the right one to approve and sign-off. I'd wait for experts on this area to chime in.
JoshRosen
left a comment
There was a problem hiding this comment.
If a cancelled task is getting stuck in the unroll while loop and isn't exiting in a timely manner then that must imply that neither the upstream values.next() calculation nor any memory reservation / spill calls are performing IO or are calling TaskContext.killTaskIfInterrupted() because otherwise the task would have been killed. In other words, there's no interruptible code running in the loop, hence the need to manually check for interrupts.
An alternative fix could be to wrap an InterruptibleIterator call around the upstream iterator, but that's much more likely to have adverse performance impacts per the linked comment because checking Thread.interrupted is cheaper than context.killTaskIfInterrupted.
Given that, I'm okay with adding logic to check Thread.interrupt, but I'm slightly wary of this PR's current approach of returning a value on interrupt: if we return Left(unrollMemoryUsedByThisBlock) then that doesn't directly cause the task to exit: it will continue running until it hits killTaskIfInterrupted() (which isn't guaranteed to be present) or until it tries to perform IO and neither of those operations are guaranteed to happen in a timely fashion (although in practice they probably will).
Instead, what do you think about replacing the Left(unrollMemoryUsedByThisBlock) with a throw new InterruptedException()? That exception will bubble up and help the task to exit sooner.
@JoshRosen - I was going to do exactly this initially :) But I was not sure whether this would be entirely safe or not, in terms of disposing the byte buffers. It seems the caller relies on the result here to create And within this, we add a task completion listener to ensure that we call dispose on the byte buffers ? So it seems freeing the memory is not a problem, but if we return the |
If I understand correctly, I think this might be a pre-existing risk that we're making worse: there's nothing that prevented the old code from throwing arbitrary exceptions when computing the iterator elements. I wonder whether we should aim to fix that pre-existing bug at a higher level. In spark/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala Lines 349 to 367 in e72ce91 I'm wondering whether we can restructure that code in order to wrap the val putIteratorResult = Utils.tryWithSafeFinallyAndFailureCallbacks {
putIterator(blockId, values, classTag, memoryMode, valuesHolder)
}(catchBlock = {
// We want to close the output stream in order to free any resources associated with the
// serializer itself (such as Kryo's internal buffers). close() might cause data to be
// written, so redirect the output stream to discard that data.
valuesHolder.redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream())
valuesHolder.serializationStream.close()
valuesHolder.bbos.close()
valuesHolder.bbos.toChunkedByteBuffer.dispose()
})
putIteratorResult match {
case Right(storedSize) => Right(storedSize)
case Left(unrollMemoryUsedByThisBlock) =>
Left(new PartiallySerializedBlock(
this,
serializerManager,
blockId,
valuesHolder.serializationStream,
valuesHolder.redirectableStream,
unrollMemoryUsedByThisBlock,
memoryMode,
valuesHolder.bbos,
values,
classTag))
}If the I scoped the As I look further into the pre-existing code, I'm spotting a couple of other cases where it looks like we're not guaranteed to perform proper cleanup. For example, it looks like we're not guaranteed to close the serialization stream if downstream partial unrolling code fails (or at least it's not straightforwardly obvious that cleanup will happen). To better test those cases, I think we should add some new unit test cases to |
|
@JoshRosen - agreed, don't think its being handled currently anyway. If we want to add the catch block on the caller, I think we need to do it for |
I don't think we need it there:
As a compromise, what do you think about keeping this as-is (i.e. not throwing val res = putIteratorResult match {
case Right(storedSize) => Right(storedSize)
case Left(unrollMemoryUsedByThisBlock) =>
Left(new PartiallySerializedBlock(
this,
serializerManager,
blockId,
valuesHolder.serializationStream,
valuesHolder.redirectableStream,
unrollMemoryUsedByThisBlock,
memoryMode,
valuesHolder.bbos,
values,
classTag))
}
Option(TaskContext.get()).foreach(_.killTaskIfInterrupted())
resThis approach will ensure that the bytebuffer cleanup logic is run (because the task completion callback will have been registered) but also ensures that we'll exit in a timely manner rather than trying to continue processing the rest of the task's rows. |
Done |
| // Unroll this block safely, checking whether we have exceeded our threshold periodically | ||
| while (values.hasNext && keepUnrolling) { | ||
| // and if no thread interrupts have been received. | ||
| while (values.hasNext && keepUnrolling && !Thread.currentThread().isInterrupted) { |
There was a problem hiding this comment.
I am slightly wary of calling isInterrupted for every iteration.
Given we already have the if (elementsUnrolled ... condition below, do we want to move it to that ?
IIRC isInterrupted is a couple of orders more expensive than a boolean check.
Something like:
var interrupted = false
while (values.hasNext && keepUnrolling && !interrupted) {
valuesHolder ...
if (elementsUnrolled ... ){
interrupted = Thread.currentThread().isInterrupted
...
}
elementsUnrolled += 1
}
if (interrupted) {
...
} else if (keepUnrolling) {
There was a problem hiding this comment.
@mridulm - if we put this check inside the if condition below, it seems we will only set the interrupted flag the next time the memory check period expires ? Seems like the default value for the check period is set to 16. So, if the interrupt is received after the first element is unrolled, we will wait for 15 more elements and also do extra work in the interim, that we probably have to dispose later anyway ? So might be better to check within the while loop and exit earlier as we are doing in the PR ?
valuesHolder.storeValue(values.next())
if (elementsUnrolled % memoryCheckPeriod == 0) {
val currentSize = valuesHolder.estimatedSize()
// If our vector's size has exceeded the threshold, request more memory
There was a problem hiding this comment.
IIRC isInterrupted is a couple of orders more expensive than a boolean check.
In older JDKs it is implemented via an intrinsic and in newer ones (14+) it's just a volatile boolean check (see https://bugs.openjdk.org/browse/JDK-8229516), so I don't think it's too expensive. There's an old StackOverflow answer at https://stackoverflow.com/a/5158441/590203 offering a plausible explanation for why the intrinsic should be cheap.
On JDK 11 on my laptop, I ran a toy JMH benchmark and measured < 1ns per call.
I think these costs are very small in comparison to the per-element serialization costs.
Given this, I'm not too worried about performance regressions due to this change and think it's probably okay to check on every element rather than every 16th.
There was a problem hiding this comment.
@anishshri-db Thread interruption is a rare occurrence when compared to number of putIterator calls - so we should try to minimize the impact for the common case, as long as it is a reasonable additional cost when interruption does occur (having said that, please see more below).
@JoshRosen It is way faster than I expected it to be [1] - while it is still slower than a local variable [2], the cost is really low when compared to the cost of other operations in the loop ... this should be just noise.
Given this, while in general I prefer to minimize unnecessary cost, I am fine with the change.
[1] Thanks for your comments, went digging into this - was fun !
I knew it was intrinsic, but the uncontended reads are much faster than I had expected (I seem to be misremembering some stats) - and thanks for jdk14 ref, was not aware of that change.
My comment "IIRC isInterrupted is a couple of orders more expensive than a boolean check." is definitely incorrect !
[2]
On my linux desktop: 2.175 +- 0.001 ns/op versus 2.424 +- 0.001 ns/op
On my mac, the difference is higher - but so is the variance ... so I am discounting that
mridulm
left a comment
There was a problem hiding this comment.
The proposal looks good to me - just a minor suggestion
|
@JoshRosen - any more comments here ? or do you think we can merge this ? |
|
@JoshRosen - seems like we are all in agreement ? Is it ok to merge the change ? Thx cc- @HeartSaVioR |
|
I'm traveling without a laptop right now, so I won't be able to merge this until next week. |
|
Thanks @JoshRosen . @mridulm , @HeartSaVioR - would one of you be able to merge then ? |
|
As long as I see @mridulm is OK with the fix, I can help merging this. Thanks! Merging to master. |
|
Thanks @HeartSaVioR ! |
What changes were proposed in this pull request?
Allow block manager memory store iterator to handle thread interrupt and perform task completion gracefully
Why are the changes needed?
Currently the
putIteratorAsBytescan remain hung even if thread interrupt is received on task cancellation leading to the task reaper killing the executor JVM eventually. The reason for this is that the interrupt is never processed within the while loop for the unroll block which leads to the task continuing running beyond the reaper timeout.Attached the logs here for a particular task/thread:
We can see that the stack traces are different over time but still within the while loop for putIterator.
Does this PR introduce any user-facing change?
No
How was this patch tested?
Ran MemoryStoreSuite
Was this patch authored or co-authored using generative AI tooling?
No