-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-39696][CORE] Ensure Concurrent r/w TaskMetrics
not throw Exception
#37206
Conversation
The exceptions reported in SPARK-39696 are as follows:
It seems to be a small probability event |
} | ||
} | ||
|
||
val writeThread1 = new Thread() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Two write threads are used to easily reproduce the issue
Have not looked in detail, but there are a bunch of other places where |
You're right. I need to rethink this problem, let me set this PR to draft first, thanks |
I'm finding the issue to be quite easy to reproduce. I have a 2 node Spark cluster with 3 executors per node (5 CPU/threads per executor). Since Friday I have ~150 DEAD executor instances due to this issue. Thanks for taking on the ticket. |
@smcmullan-ncirl I wonder if there will be such a high frequency of failures when using Scala 2.12? |
I'm a novice with the Spark source code base and the Scala source code base but it looks to me that the scala.collection.mutable.MutationTracker class and scala.collection.mutable.CheckedIndexedSeqView class only exists in Scala 2.13 so I'm guessing its a case that with Java 17 JVM and Scala 2.13 that I'm running on a most restrictive runtime platform compared to the majority of Spark users |
Yes. When running new test suite, |
TaskMetrics
not throw ExceptionTaskMetrics
not throw Exception
@mridulm Compared with analyzing each scenario and using read-write locks, I think it may be simpler to change |
I agree, given the usecase, |
TaskMetrics
not throw ExceptionTaskMetrics
not throw Exception
@mridulm, thanks for the ping.
It looks like In the SPARK-39696 JIRA, it looks like the read is occurring in the executor's
As far as I know, spark/core/src/main/scala/org/apache/spark/executor/Executor.scala Lines 507 to 508 in 0cc96f7
Assuming my above description is correct, I don't understand how concurrent reads and writes can occur: in normal operation I would expect that writes only occur during task binary deserialization and that reads from the heartbeat thread can only occur after deserialization has completed. Perhaps accumulators are being deserialized and registered while the task is running (not just during task setup)? For example, maybe a case class is defined on the driver and its closure accidentally closes over something that contains an AccumulatorV2, causing us to deserialize AccumulatorV2 instances when deserializing data values. In the simple case, we'd have one writer thread (the task's main runner thread) racing with the heartbeat thread. In more complex cases, I can imagine scenarios where a single task could consist of multiple threads (e.g. for PySpark) and deserialization might happen in multiple of them in case of spill (e.g. during RDD aggregation). If this is the case then it might explain the race condition. If that's true, though, then I'm wondering whether we're registering different instances of the same accumulator multiple times in the same task (which seems like it would be a performance and correctness bug). |
Yes, your analysis is very accurate. From the current stack, I can only infer that the following two methods may have racing (but I haven't found any conclusive evidence), so I added read-write locks to these two methods in the initial pr. spark/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala Lines 257 to 261 in 66b1f79
I didn't expect there would be correctness bug here, if this is really possible, I think the current clues are not enough to troubleshoot the problem. @smcmullan-ncirl Cloud you provide more details to further investigate the issue? I want to know what the writing thread is?It seems that only you can stably reproduce this problem now :) |
Thanks for the detailed analysis @JoshRosen - I agree with your analysis.
Agree, we should get more details on this before changing the type/other fixes. |
Agree +1, we need more details |
Thanks for all the analysis and effort. I think what @JoshRosen is describing in the last paragraph of his analysis is exactly what my application is doing. I have a custom ForeachWriter implementation class acting as a data sink on the write stream and I'm passing an instance of a statistics gathering class as an argument to the ForeachWriterImpl constructor. This statistics class has several Maps which have keys as statistic name and the value is a Spark accumulator. I think I have some defects in my application where the de-serialization of this statistics class instance on the executor is re-registering the accumulator in the Spark context as described in the analysis above. I will try to reorganize my code and see if I can stop the issue happening. If you need to see more details I can probably code up a sample application to show the construction of my production application |
@smcmullan-ncirl Any new progress? |
Hi, yes and no. I produced this toy example of what I thought my production code is doing: https://github.com/smcmullan-ncirl/RateSourceSparkStreamTest However it seems to work fine and has not reproduced the error or caused any executors to crash. Maybe the code will give some clues as to whether I'm using some bad practice which exposes the issue in the production code. I'm still trying to analyze the production code to see where there may be differences with the toy application. I'll report back soon if I find any or if I'm able to extend the toy application to reproduce the error |
Hi, so I was able to reproduce the issue with the example application below. It shows an custom ForEach sink which updates a set of Accumulators. The Map containing the accumulators is passed in the call to the Foreach sink and thus serialiazed/deserialized I've integrated Codahale/DropWizard metrics for reporting metrics based on the accumulators and enabled the metrics sink like this:
You may notice in the example application code below that that overridden method for getMetrics() resets the accumulator by calling its reset() method. I wonder whether this causes the situation that @JoshRosen wrote about above?
|
@LuciferYang - does the above make any sense? Do you need any more information from me? |
Hi! |
I don't have any new ideas at present |
No better idea, close it first |
Hi - our executors crashed almost every 30 min while using HDFS as checkpoint (Kafka source). FYI - using SSHFS as checkpoint would crash the executors randomly say once/twice per day... From our opinion, this is too dangerous in production, as this would cause streaming jitters - thus causing unncessary backpressure on our data pipeline. ... We intend to fallback to Spark-3.3.1 Scala-2.12 (instead of Scala 2.13) Is there a Spark LTS release that is stable for production use? |
Yes - Fallback to Scala 2.12 avoids this issue. For 2 hours under heavy stress-load, there is no issue (as of now yet...) with HDFS checkpointing. Our UAT setup is RHEL/8.6, 3-nodes, Spark 3.3.1 (hadoop3 + scala-2.12) |
@LuciferYang FYI ^ |
Although I'm not quite sure, this sounds more like a issue of Scala 2.13.8 itself |
|
There is no issue with Scala 2.12 at all for rhe entire week. |
@LuciferYang FYI
|
We are also seeing this failure on Spark 3.3.1 with Scala 2.13 on Ubuntu 22.04. I used one of the spark applications seeing this issue to further debug when we are deserializing If the which is called after the Task deserilization happens. This combined with a with a the our I would be willing to work no a fix for this, but would probably need some guidance/discussion what the correct fix actually is. |
I would kindly ask for this feature to be disabled by default - before proven stable for Scala 2.13. Otherwise, this would hurt its reputation |
@tamama Please update the jira with details that can help debug the issue - specifically, the test snippet you used to surface issue in 2.13 and which does not occur in 2.12 |
@eejbyfeldt shuffle write processor is an instance variable, and so is a part of the dependency created at driver |
While that statement is true. That is not relevant for the point I was trying to make. @JoshRosen commented in #37206 (comment) claiming that we should not perform concurrent access due to the fact that accumulators should be deserialized and therefore registered during the Task deserilization and there should be no race with the hearbeat thread. But my claim was that is not true as accumulators will be deserialized in the |
…cums ### What changes were proposed in this pull request? This PR fixes a data race around concurrent access to `TaskMetrics.externalAccums`. The race occurs between the `executor-heartbeater` thread and the thread executing the task. This data race is not known to cause issues on 2.12 but in 2.13 ~due this change scala/scala#9258 (LuciferYang bisected this to first cause failures in scala 2.13.7 one possible reason could be scala/scala#9786) leads to an uncaught exception in the `executor-heartbeater` thread, which means that the executor will eventually be terminated due to missing hearbeats. This fix of using of using `CopyOnWriteArrayList` is cherry picked from #37206 where is was suggested as a fix by LuciferYang since `TaskMetrics.externalAccums` is also accessed from outside the class `TaskMetrics`. The old PR was closed because at that point there was no clear understanding of the race condition. JoshRosen commented here #37206 (comment) saying that there should be no such race based on because all accumulators should be deserialized as part of task deserialization here: https://github.com/apache/spark/blob/0cc96f76d8a4858aee09e1fa32658da3ae76d384/core/src/main/scala/org/apache/spark/executor/Executor.scala#L507-L508 and therefore no writes should occur while the hearbeat thread will read the accumulators. But my understanding is that is incorrect as accumulators will also be deserialized as part of the taskBinary here: https://github.com/apache/spark/blob/169f828b1efe10d7f21e4b71a77f68cdd1d706d6/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala#L87-L88 which will happen while the heartbeater thread is potentially reading the accumulators. This can both due to user code using accumulators (see the new test case) but also when using the Dataframe/Dataset API as sql metrics will also be `externalAccums`. One way metrics will be sent as part of the taskBinary is when the dep is a `ShuffleDependency`: https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/core/src/main/scala/org/apache/spark/Dependency.scala#L85 with a ShuffleWriteProcessor that comes from https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala#L411-L422 ### Why are the changes needed? The current code has a data race. ### Does this PR introduce _any_ user-facing change? It will fix an uncaught exception in the `executor-hearbeater` thread when using scala 2.13. ### How was this patch tested? This patch adds a new test case, that before the fix was applied consistently produces the uncaught exception in the heartbeater thread when using scala 2.13. Closes #40663 from eejbyfeldt/SPARK-39696. Lead-authored-by: Emil Ejbyfeldt <[email protected]> Co-authored-by: yangjie01 <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
…cums ### What changes were proposed in this pull request? This PR fixes a data race around concurrent access to `TaskMetrics.externalAccums`. The race occurs between the `executor-heartbeater` thread and the thread executing the task. This data race is not known to cause issues on 2.12 but in 2.13 ~due this change scala/scala#9258 (LuciferYang bisected this to first cause failures in scala 2.13.7 one possible reason could be scala/scala#9786) leads to an uncaught exception in the `executor-heartbeater` thread, which means that the executor will eventually be terminated due to missing hearbeats. This fix of using of using `CopyOnWriteArrayList` is cherry picked from #37206 where is was suggested as a fix by LuciferYang since `TaskMetrics.externalAccums` is also accessed from outside the class `TaskMetrics`. The old PR was closed because at that point there was no clear understanding of the race condition. JoshRosen commented here #37206 (comment) saying that there should be no such race based on because all accumulators should be deserialized as part of task deserialization here: https://github.com/apache/spark/blob/0cc96f76d8a4858aee09e1fa32658da3ae76d384/core/src/main/scala/org/apache/spark/executor/Executor.scala#L507-L508 and therefore no writes should occur while the hearbeat thread will read the accumulators. But my understanding is that is incorrect as accumulators will also be deserialized as part of the taskBinary here: https://github.com/apache/spark/blob/169f828b1efe10d7f21e4b71a77f68cdd1d706d6/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala#L87-L88 which will happen while the heartbeater thread is potentially reading the accumulators. This can both due to user code using accumulators (see the new test case) but also when using the Dataframe/Dataset API as sql metrics will also be `externalAccums`. One way metrics will be sent as part of the taskBinary is when the dep is a `ShuffleDependency`: https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/core/src/main/scala/org/apache/spark/Dependency.scala#L85 with a ShuffleWriteProcessor that comes from https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala#L411-L422 ### Why are the changes needed? The current code has a data race. ### Does this PR introduce _any_ user-facing change? It will fix an uncaught exception in the `executor-hearbeater` thread when using scala 2.13. ### How was this patch tested? This patch adds a new test case, that before the fix was applied consistently produces the uncaught exception in the heartbeater thread when using scala 2.13. Closes #40663 from eejbyfeldt/SPARK-39696. Lead-authored-by: Emil Ejbyfeldt <[email protected]> Co-authored-by: yangjie01 <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]> (cherry picked from commit 6ce0822) Signed-off-by: Hyukjin Kwon <[email protected]>
…cums ### What changes were proposed in this pull request? This PR fixes a data race around concurrent access to `TaskMetrics.externalAccums`. The race occurs between the `executor-heartbeater` thread and the thread executing the task. This data race is not known to cause issues on 2.12 but in 2.13 ~due this change scala/scala#9258 (LuciferYang bisected this to first cause failures in scala 2.13.7 one possible reason could be scala/scala#9786) leads to an uncaught exception in the `executor-heartbeater` thread, which means that the executor will eventually be terminated due to missing hearbeats. This fix of using of using `CopyOnWriteArrayList` is cherry picked from #37206 where is was suggested as a fix by LuciferYang since `TaskMetrics.externalAccums` is also accessed from outside the class `TaskMetrics`. The old PR was closed because at that point there was no clear understanding of the race condition. JoshRosen commented here #37206 (comment) saying that there should be no such race based on because all accumulators should be deserialized as part of task deserialization here: https://github.com/apache/spark/blob/0cc96f76d8a4858aee09e1fa32658da3ae76d384/core/src/main/scala/org/apache/spark/executor/Executor.scala#L507-L508 and therefore no writes should occur while the hearbeat thread will read the accumulators. But my understanding is that is incorrect as accumulators will also be deserialized as part of the taskBinary here: https://github.com/apache/spark/blob/169f828b1efe10d7f21e4b71a77f68cdd1d706d6/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala#L87-L88 which will happen while the heartbeater thread is potentially reading the accumulators. This can both due to user code using accumulators (see the new test case) but also when using the Dataframe/Dataset API as sql metrics will also be `externalAccums`. One way metrics will be sent as part of the taskBinary is when the dep is a `ShuffleDependency`: https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/core/src/main/scala/org/apache/spark/Dependency.scala#L85 with a ShuffleWriteProcessor that comes from https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala#L411-L422 ### Why are the changes needed? The current code has a data race. ### Does this PR introduce _any_ user-facing change? It will fix an uncaught exception in the `executor-hearbeater` thread when using scala 2.13. ### How was this patch tested? This patch adds a new test case, that before the fix was applied consistently produces the uncaught exception in the heartbeater thread when using scala 2.13. Closes #40663 from eejbyfeldt/SPARK-39696. Lead-authored-by: Emil Ejbyfeldt <[email protected]> Co-authored-by: yangjie01 <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]> (cherry picked from commit 6ce0822) Signed-off-by: Hyukjin Kwon <[email protected]> (cherry picked from commit b2ff4c4) Signed-off-by: Dongjoon Hyun <[email protected]>
…cums ### What changes were proposed in this pull request? This PR fixes a data race around concurrent access to `TaskMetrics.externalAccums`. The race occurs between the `executor-heartbeater` thread and the thread executing the task. This data race is not known to cause issues on 2.12 but in 2.13 ~due this change scala/scala#9258 (LuciferYang bisected this to first cause failures in scala 2.13.7 one possible reason could be scala/scala#9786) leads to an uncaught exception in the `executor-heartbeater` thread, which means that the executor will eventually be terminated due to missing hearbeats. This fix of using of using `CopyOnWriteArrayList` is cherry picked from apache#37206 where is was suggested as a fix by LuciferYang since `TaskMetrics.externalAccums` is also accessed from outside the class `TaskMetrics`. The old PR was closed because at that point there was no clear understanding of the race condition. JoshRosen commented here apache#37206 (comment) saying that there should be no such race based on because all accumulators should be deserialized as part of task deserialization here: https://github.com/apache/spark/blob/0cc96f76d8a4858aee09e1fa32658da3ae76d384/core/src/main/scala/org/apache/spark/executor/Executor.scala#L507-L508 and therefore no writes should occur while the hearbeat thread will read the accumulators. But my understanding is that is incorrect as accumulators will also be deserialized as part of the taskBinary here: https://github.com/apache/spark/blob/169f828b1efe10d7f21e4b71a77f68cdd1d706d6/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala#L87-L88 which will happen while the heartbeater thread is potentially reading the accumulators. This can both due to user code using accumulators (see the new test case) but also when using the Dataframe/Dataset API as sql metrics will also be `externalAccums`. One way metrics will be sent as part of the taskBinary is when the dep is a `ShuffleDependency`: https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/core/src/main/scala/org/apache/spark/Dependency.scala#L85 with a ShuffleWriteProcessor that comes from https://github.com/apache/spark/blob/fbbcf9434ac070dd4ced4fb9efe32899c6db12a9/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala#L411-L422 ### Why are the changes needed? The current code has a data race. ### Does this PR introduce _any_ user-facing change? It will fix an uncaught exception in the `executor-hearbeater` thread when using scala 2.13. ### How was this patch tested? This patch adds a new test case, that before the fix was applied consistently produces the uncaught exception in the heartbeater thread when using scala 2.13. Closes apache#40663 from eejbyfeldt/SPARK-39696. Lead-authored-by: Emil Ejbyfeldt <[email protected]> Co-authored-by: yangjie01 <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]> (cherry picked from commit 6ce0822) Signed-off-by: Hyukjin Kwon <[email protected]>
@JoshRosen, @eejbyfeldt , I hope it is fine that I am posting here after the ticket has been closed. We caught a problem which, I believe, has the same 'true' root cause as in this ticket, and I also think that the 'true' root cause was not eliminated by the fix. I think that 'ConcurrentModificationException' is only one of the possible outcomes in the various race conditions scenarios (concurrent reads/writes). Particularly, I managed to catch NullPointerException (instead of ConcurrentModificationException) in my custom accumulator where I would never expect it to be caught. Below is a simple reproducer for demonstrating the bigger problem:
The code listed above should be executed from the spark-shell, started with spark.executor.heartbeatInterval=1 (ms) in order to increase the chances of the appearance of the issue (on my laptop it is enough to call tryReproduce() just once):
The NullPointerException from the 'executor-heartbeater' thread looks like this:
In my real-world case, my accumulator has a final Java field which can never be null... Therefore, I suspect there is some lack of synchronization between the deserialization code and accessing the 'isZero' method of a just deserialized accumulator instance. And, therefore, I think replacing ArrayBuffer by CopyOnWriteArrayList does not completely fix the bigger issue. Also, in my real-world case we have default 10s for spark.executor.heartbeatInterval. Still, somehow we managed to catch many NullPointerExceptions from the 'executor-heartbeater' thread over 1 day, resulting in executors re-started (which led to unbalanced load distribution in a Spark Standalone cluster). |
Hi @gowa, I don't think you issue you describe has the same root cause and I think it more related to how serialization work. Consider the following code:
when executed it will print
This code is similar to what we have in Here is a PR addressing/discussing the same bug from an accumulator in spark: #31540 |
Hi @eejbyfeldt ! Oh, thank you! it is clear to me now. I am sorry for disturbing. Thank you very much! |
What changes were proposed in this pull request?
This PR changes the declaration type of
TaskMetrics#externalAccums
froms.c.mutable.ArrayBuffer
toj.u.concurrent.CopyOnWriteArrayList
to ensure that errors described in SPARK-39696(java.util.ConcurrentModificationException: mutation occurred during iteration
) will not occur whenTaskMetrics#externalAccums
is read and written concurrentlyWhy are the changes needed?
Bug fix.
Does this PR introduce any user-facing change?
No
How was this patch tested?
Pass GitHub Actions and add a new test case.
The new case will fail regardless of the Scala version without changes of
TaskMetrics
as follows:Scala 2.12:
Before
After
Scala 2.13
Before
After