-
Notifications
You must be signed in to change notification settings - Fork 236
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
[DISCUSS] Shuffle read-side error handling #326
Comments
Looks good to me. |
Filed this (#739) to handle zero-sized blocks, local block failures, and compression issues. |
The worry here I think is somewhat unfounded. It's not that we'll run out of device memory, but instead we'll have to spill and track buffers that will never be used, so yes we'll ultimately cause more memory pressure and disk usage. We also could have pending transfers for these blocks, so we will want to handle them. Filed #742 to track. |
There is already an issue to try and add more tests for partitioning: #424. We could add here some of the partitioning tests that spark has, for bound computation (as suggested by @kuhushukla). No issue to report, just something we should have tests for. |
Signed-off-by: spark-rapids automation <[email protected]>
Currently, any errors from the RapidsShuffleClient would cause an
IllegalStateException
, triggering anExecutor
failure (as this is a fatal exception). In our case, this means a new JVM would have to launch and take hold of the GPU.Shutting down the
Executor
process can cause peers waiting on pending UCX transfers to fail, as theExecutor
is also a shuffle server, so we want to avoid exiting the process as much as possible.The following is a plan to use FetchFailedException when appropriate.
The end result of this ticket should be documentation on how retried tasks are handled in the Rapids plugin, specifically with the shuffle component as it deals with map output caching, and code changes to ensure correctness (if any), and improve robustness.
How
FetchFailedException
is handled in Spark:When constructing a new
FetchFailedException
, the TaskContext‘ssetFetchFailed
is called, recording a single fetch failure for a(shuffleId, mapId, mapIndex, reduceId)
tuple. On callingnext()
on the shuffle iterator, this exception is thrown. Only a single exception is really observed. In Spark proper, if multiple tasks were running and corrupted, then we’ll fail again when we detect the new failure.There is also a MetadataFetchFailedException. This is used to retry the task when there is a failure fetching from the
MapOutputTracker
On the various ids (for reference, more eyes on this could be great):
shuffleId
is part of the ShuffleDependency.mapId
can be different things depending on the shuffle protocol used. This is decided in ShuffleMapTask.partitionId
. This number comes from theDAGScheduler
which queries the Job for the partitions that are pending (not finished). This is found byfindMissingPartitions
in the shuffle stage, forShuffleMap
stages. This is an index into theMapStatus
array.taskAttemptId
([SPARK-25341][Core] Support rolling back a shuffle map stage and re-generate the shuffle files apache/spark#25620, [SPARK-29435][Core]MapId in Shuffle Block is inconsistent at the writer and reader part when spark.shuffle.useOldFetchProtocol=true apache/spark#26095) This number is unique per partition/stage and does change between attempts. It comes from theTaskSchedulerImpl.newTaskId
method, called from theTaskSetManager
’sdequeueTask
handler, which increments an atomic long member ofTaskSchedulerImpl
(this happens in the driver)BlockId
? That would bemapId
.mapIndex
is not the same asmapId
. See convertMapStatuses inMapOutputTracker
. It is the old mappartitionId
, so if there are 1000 map tasks, it is a number between 0 and 999.reduceId
is the reduce partition produced by the mapper. It is stored inMapStatus
(sizes array).Executive summary:
On a
FetchFailedException
, Spark handles it by resubmitting certain stages. Some stages are not fully submitted, but instead, Spark, given the failed map stage and themapIndex
(encoded in theFetchFailedException
), will fail it, unregister the map output, and any indeterminate stage that it depends on, because its output is non deterministic, must be recomputed. Additionally, theDAGScheduler
will unregister all of the executor's output..Specifics:
DAGScheduler
receives aFetchFailed
event from theExecutor
(who had a try/catch around the task, and caught theFetchFailedException
).failedAttemptIds
inStage.scala
) up tospark.stage.maxConsecutiveAttempts
(4 by default).mapIndex
is unregistered (shuffleId
,mapIndex
, block manager)reduceByKey
, we should make sure that theRapidsShuffleManager
at least blows up if it sees aShuffleDependency
that has these requirements.blockManagerId
passed toFetchFailedException
is specified,removeOutputsOnExecutor
is called, clearing any map output that the peer executor had computed.Question: Are map indices (partitions) always deterministic? And if not, are we making sure the RDD is marked non deterministic
When does Spark throw
FetchFailedException
:FetchFailedException
is thrown.Map Output Corruption
There has been plenty of work around how to deal with map output corruption (https://issues.apache.org/jira/browse/SPARK-8029). In the cases outlined, the issues are normally related to multiple live copies of a map task trying to commit output, due to speculation, or due to re-run caused by a fetch failure.
Presently, outputs have little chance of corrupting each other. Speculative tasks are tried on different hosts and also mean that each will get their own
taskAttemptId
, retried tasks will have differenttaskAttemptId
s under the new shuffle protocol) and committing is "last writer wins".Question: I believe our real issue is running out of memory. We will happily write the executor's blocks again to the catalog, but there is no callback from the
MapOutputTracker
saying we can forget those old blocks we are retrying. There may be enough hooks to do this, but I don't think we do now.Current change proposal:
RapidsShuffleIterator
will throw aRapidsShuffleFetchFailedException
when there was a fetch failure and the caller calls next. The expectation is that the shuffle stage will be re-submitted, along the map stage that computed the failed partition, given apache/spark#18150 all map output for the executor that we were fetching from, and any of their dependents as perDAGScheduler
requirements. We'll complain about all the blocks the peer was responsible for at first, without changing the protocol. We can change the protocol to preserve themapIndex
, which would allow us to point at a particular bad partition in the future (since all map outputs for the executor will be recomputed, it doesn't matter much in those terms).The error comes from a
TransactionStatus.Error
orTransactionStatus.Cancelled
. Currently, they are handled the same way. Cancelled transactions are likely caused by an error (while tearing down UCX classes). In the future cancelled vs error may have two different error handling paths. This should handle mostly fetch failures, and other PRs should follow if we agree.I don't have a change that would deal with the old retried accumulating map outputs. We can update this (and the potential doc with some of this info) when that is ready.
Error examples:
Metadata fetch failure:
Buffer fetch failure:
DAG scheduler example showing the handling:
DAG scheduler removes the executor output altogether:
Later the executor is told to re-register
And it gets re-added
Executor 27 only shuts down when the job finishes:
Note: this does resubmit stage 21 and 22 (fetch error happened in 23), and then later 23 executes. It works well, unless there are other fetch failures that are not triggered by the rapids shuffle client (see below)
Bad case:
In this case, it’s not just the two failures (which were induced for executor id 1 and 2). There are other failures when a map status is missing. These lead to job failures, and don’t happen all the time.
A
mapIndex
of -1 normally means a metadata fetch failure, from the code I've seen. This last case needs to be looked at more.--- UPDATE ---
Timeout handling
While fetching metadata, buffers, or connecting to another client, we would like to be able to time out and either fail the job or retry.
As of this update, #732 handles metadata + buffer fetch timeouts in a basic way, but it serves the purpose of not having a user wait indefinitely. UCX is looking to improve error handling features and this is an area where we could get finer grain errors.
The text was updated successfully, but these errors were encountered: