Skip to content
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-34939][CORE][2.4] Throw fetch failure exception when unable to deserialize broadcasted map statuses #32045

Closed
wants to merge 1 commit into from

Conversation

viirya
Copy link
Member

@viirya viirya commented Apr 3, 2021

What changes were proposed in this pull request?

This patch catches IOException, which is possibly thrown due to unable to deserialize map statuses (e.g., broadcasted value is destroyed), when deserilizing map statuses. Once IOException is caught, MetadataFetchFailedException is thrown to let Spark handle it.

This is a backport of #32033 to branch-2.4.

Why are the changes needed?

One customer encountered application error. From the log, it is caused by accessing non-existing broadcasted value. The broadcasted value is map statuses. E.g.,

[info]   Cause: java.io.IOException: org.apache.spark.SparkException: Failed to get broadcast_0_piece0 of broadcast_0                                                                                                                  
[info]   at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1410)                                                                                                                                                            
[info]   at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:226)                                                                                                                                 
[info]   at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:103)                                                                                                                                           
[info]   at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)                                                                                                                                                             
[info]   at org.apache.spark.MapOutputTracker$.$anonfun$deserializeMapStatuses$3(MapOutputTracker.scala:967)                                                                                                                           
[info]   at org.apache.spark.internal.Logging.logInfo(Logging.scala:57)                                                                                                                                                                
[info]   at org.apache.spark.internal.Logging.logInfo$(Logging.scala:56)                                                                                                                                                               
[info]   at org.apache.spark.MapOutputTracker$.logInfo(MapOutputTracker.scala:887)                                                                                                                                                     
[info]   at org.apache.spark.MapOutputTracker$.deserializeMapStatuses(MapOutputTracker.scala:967)                                                                                                                                                                                                                                                                                 

There is a race-condition. After map statuses are broadcasted and the executors obtain serialized broadcasted map statuses. If any fetch failure happens after, Spark scheduler invalidates cached map statuses and destroy broadcasted value of the map statuses. Then any executor trying to deserialize serialized broadcasted map statuses and access broadcasted value, IOException will be thrown. Currently we don't catch it in MapOutputTrackerWorker and above exception will fail the application.

Normally we should throw a fetch failure exception for such case. Spark scheduler will handle this.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Unit test.

@SparkQA
Copy link

SparkQA commented Apr 3, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41462/

@SparkQA
Copy link

SparkQA commented Apr 3, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41462/

@viirya
Copy link
Member Author

viirya commented Apr 3, 2021

Thank you @dongjoon-hyun

@SparkQA
Copy link

SparkQA commented Apr 3, 2021

Test build #136886 has finished for PR 32045 at commit 62ec191.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

dongjoon-hyun pushed a commit that referenced this pull request Apr 4, 2021
… deserialize broadcasted map statuses

### What changes were proposed in this pull request?

This patch catches `IOException`, which is possibly thrown due to unable to deserialize map statuses (e.g., broadcasted value is destroyed), when deserilizing map statuses. Once `IOException` is caught, `MetadataFetchFailedException` is thrown to let Spark handle it.

This is a backport of #32033 to branch-2.4.

### Why are the changes needed?

One customer encountered application error. From the log, it is caused by accessing non-existing broadcasted value. The broadcasted value is map statuses. E.g.,

```
[info]   Cause: java.io.IOException: org.apache.spark.SparkException: Failed to get broadcast_0_piece0 of broadcast_0
[info]   at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1410)
[info]   at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:226)
[info]   at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:103)
[info]   at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
[info]   at org.apache.spark.MapOutputTracker$.$anonfun$deserializeMapStatuses$3(MapOutputTracker.scala:967)
[info]   at org.apache.spark.internal.Logging.logInfo(Logging.scala:57)
[info]   at org.apache.spark.internal.Logging.logInfo$(Logging.scala:56)
[info]   at org.apache.spark.MapOutputTracker$.logInfo(MapOutputTracker.scala:887)
[info]   at org.apache.spark.MapOutputTracker$.deserializeMapStatuses(MapOutputTracker.scala:967)
```

There is a race-condition. After map statuses are broadcasted and the executors obtain serialized broadcasted map statuses. If any fetch failure happens after, Spark scheduler invalidates cached map statuses and destroy broadcasted value of the map statuses. Then any executor trying to deserialize serialized broadcasted map statuses and access broadcasted value, `IOException` will be thrown. Currently we don't catch it in `MapOutputTrackerWorker` and above exception will fail the application.

Normally we should throw a fetch failure exception for such case. Spark scheduler will handle this.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit test.

Closes #32045 from viirya/fix-broadcast.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
@dongjoon-hyun
Copy link
Member

Jenkins passed. Merged to branch-2.4.

@viirya
Copy link
Member Author

viirya commented Apr 4, 2021

Thank you @dongjoon-hyun @HyukjinKwon

otterc pushed a commit to linkedin/spark that referenced this pull request Mar 22, 2023
… deserialize broadcasted map statuses

This patch catches `IOException`, which is possibly thrown due to unable to deserialize map statuses (e.g., broadcasted value is destroyed), when deserilizing map statuses. Once `IOException` is caught, `MetadataFetchFailedException` is thrown to let Spark handle it.

This is a backport of apache#32033 to branch-2.4.

One customer encountered application error. From the log, it is caused by accessing non-existing broadcasted value. The broadcasted value is map statuses. E.g.,

```
[info]   Cause: java.io.IOException: org.apache.spark.SparkException: Failed to get broadcast_0_piece0 of broadcast_0
[info]   at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1410)
[info]   at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:226)
[info]   at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:103)
[info]   at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
[info]   at org.apache.spark.MapOutputTracker$.$anonfun$deserializeMapStatuses$3(MapOutputTracker.scala:967)
[info]   at org.apache.spark.internal.Logging.logInfo(Logging.scala:57)
[info]   at org.apache.spark.internal.Logging.logInfo$(Logging.scala:56)
[info]   at org.apache.spark.MapOutputTracker$.logInfo(MapOutputTracker.scala:887)
[info]   at org.apache.spark.MapOutputTracker$.deserializeMapStatuses(MapOutputTracker.scala:967)
```

There is a race-condition. After map statuses are broadcasted and the executors obtain serialized broadcasted map statuses. If any fetch failure happens after, Spark scheduler invalidates cached map statuses and destroy broadcasted value of the map statuses. Then any executor trying to deserialize serialized broadcasted map statuses and access broadcasted value, `IOException` will be thrown. Currently we don't catch it in `MapOutputTrackerWorker` and above exception will fail the application.

Normally we should throw a fetch failure exception for such case. Spark scheduler will handle this.

No

Unit test.

Closes apache#32045 from viirya/fix-broadcast.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>

(cherry picked from commit 30436b5)

RB=2855790
BUG=LIHADOOP-61824
G=spark-reviewers
R=yezhou,mmuralid,vsowrira
A=mmuralid,vsowrira
@viirya viirya deleted the fix-broadcast branch December 27, 2023 18:27
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants