Skip to content

Conversation

@nrghosh
Copy link
Contributor

@nrghosh nrghosh commented Jan 14, 2026

Why are these changes needed?

When EngineDeadError occurs, the vLLM engine subprocess is dead but the Ray actor process is still alive. Previously, we re-raised the exception, but this causes task retries to go to the SAME actor (actor methods are bound to specific instances), creating an infinite retry loop on the broken actor.

The Problem

vLLM engine subprocess crashes
       ↓
EngineDeadError raised
       ↓
Exception re-raised (actor stays ALIVE)
       ↓
Ray: actor_task_retry_on_errors triggers retry
       ↓
Retry goes to SAME actor (actor methods are bound)
       ↓
Same actor, engine still dead → EngineDeadError
       ↓
Infinite loop (with max_task_retries=-1)

The Fix

Call os._exit(1) to exit the actor. This triggers Ray to:

  1. Mark the actor as RESTARTING
  2. Create a replacement actor with a fresh vLLM engine
  3. Route task retries to healthy actors (Ray Data excludes RESTARTING actors from dispatch)

This leverages Ray Data's existing fault tolerance infrastructure:

  • max_restarts=-1 (default) enables actor replacement
  • max_task_retries=-1 (default) enables task retry

Why os._exit(1) instead of ray.actor.exit_actor()?

We must use os._exit(1) rather than ray.actor.exit_actor() because they produce different exit types with different retry behavior:

Exit Method Exit Type Exception Raised Retried?
os._exit(1) SYSTEM_ERROR RaySystemError Yes
ray.actor.exit_actor() INTENDED_USER_EXIT ActorDiedError No

The root cause is that Ray Data only adds RaySystemError to its retry_exceptions list (in _add_system_error_to_retry_exceptions()). Since ActorDiedError is NOT a subclass of RaySystemError (they're siblings in the exception hierarchy), tasks that fail due to ray.actor.exit_actor() are not retried.

The semantic gap: Ray currently lacks a "fatal application error" concept - an error where the actor should be restarted AND pending tasks retried. The available options are:

  • Clean exit (exit_actor) = "I'm intentionally done" → no retry
  • Crash (os._exit) = "Something broke unexpectedly" → retry

We need the "crash" semantics even though this is a deliberate decision, so os._exit(1) is the correct workaround until Ray Core adds explicit support for fatal application errors.

See: #60150

cc @goutamvenkat-anyscale

Validation

We created a minimal reproduction script demonstrating:

  1. The problem: All retries go to the same broken actor (same PID)
  2. The fix: Actor exits → replacement created → job succeeds with multiple PIDs
# Demo output showing fix works:
[SUCCESS] Processed 20 rows!
PIDs that processed batches: {708593, 708820}
-> Multiple PIDs = replacement actor joined and processed work

Full reproduction: https://gist.github.com/nrghosh/c18e514a975144a238511012774bab8b

Related issue number

Fixes #59522

Checks

  • I've signed off every commit
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've added any new APIs to the API Reference. For example, if I added a
    temporary file handling method, I've added it in doc/source/ray-core/api/doc/ray.util.temp_files.rst.
  • I've made sure the tests are passing. Note that there might be a few flaky tests.

@nrghosh nrghosh requested a review from a team as a code owner January 14, 2026 20:46
@nrghosh nrghosh requested review from a team and richardliaw January 14, 2026 20:47
Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Code Review

This pull request introduces a fix for handling fatal vLLM engine errors by forcefully exiting the Ray actor, which allows Ray's fault tolerance to restart it. The approach is sound and the reasoning is well-documented in the code comments.

My main feedback is to use the idiomatic Ray API ray.actor.exit_actor() instead of the low-level os._exit(1). This improves code clarity and relies on the framework's intended mechanism for actor lifecycle management. This would also involve changing the new import os to import ray.

@nrghosh nrghosh requested a review from kouroshHakha January 14, 2026 20:48
@nrghosh nrghosh added data Ray Data-related issues llm go add ONLY when ready to merge, run all tests labels Jan 14, 2026
When EngineDeadError occurs, the vLLM engine subprocess is dead but
the Ray actor process is still alive. Previously, we re-raised the
exception, but this causes task retries to go to the SAME actor
(actor methods are bound to specific instances), creating an infinite
retry loop on the broken actor.

The fix: call ray.actor.exit_actor() to exit the actor. This triggers
Ray to:
1. Mark the actor as RESTARTING
2. Create a replacement actor with a fresh vLLM engine
3. Route task retries to healthy actors (Ray Data excludes
   RESTARTING actors from dispatch)

This leverages Ray Data's existing fault tolerance infrastructure:
- max_restarts=-1 (default) enables actor replacement
- max_task_retries=-1 (default) enables task retry

Fixes ray-project#59522

Signed-off-by: Nikhil Ghosh <nikhil@anyscale.com>
@nrghosh nrghosh force-pushed the nrghosh/exit-actor-on-engine-dead-error branch from 021ca49 to a388125 Compare January 14, 2026 20:51
@nrghosh
Copy link
Contributor Author

nrghosh commented Jan 14, 2026

/gemini review

Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Code Review

This pull request addresses a critical issue where a fatal EngineDeadError from the vLLM engine could lead to an infinite retry loop on a broken actor. The proposed solution is to explicitly terminate the actor process upon such an error, enabling Ray's fault tolerance mechanisms to restart it. The implementation is clean and effective. It correctly uses ray.actor.exit_actor() to terminate the actor, which is the recommended public API for this purpose, ensuring a graceful shutdown. The added comments clearly explain the rationale behind the change, and the logging will be helpful for debugging. The change is well-contained and directly solves the described problem. Overall, this is an excellent fix.

@kouroshHakha kouroshHakha enabled auto-merge (squash) January 14, 2026 22:30
Update test_vllm_udf_fatal_error_always_raises to verify that fatal
errors (EngineDeadError) now trigger ray.actor.exit_actor() for
recovery instead of simply re-raising.

The original intent (PR ray-project#59212) was that fatal errors should NOT be
swallowed by should_continue_on_error. This is preserved - fatal
errors still don't yield error rows. The change is that instead of
re-raising (which caused infinite retry loops on the same broken
actor), we now exit the actor to enable recovery.

Signed-off-by: Nikhil Ghosh <nikhil@anyscale.com>
@github-actions github-actions bot disabled auto-merge January 14, 2026 23:47
Copy link
Contributor

@kouroshHakha kouroshHakha left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@github-actions github-actions bot disabled auto-merge January 15, 2026 01:20
ray.actor.exit_actor() doesn't work for our use case because:
- It creates INTENDED_USER_EXIT exit type
- This raises ActorDiedError (not RaySystemError)
- ActorDiedError is NOT in Ray Data's retry_exceptions list
- Therefore the task is NOT retried - job fails immediately

os._exit(1) works because:
- It creates SYSTEM_ERROR exit type
- This raises RaySystemError
- RaySystemError IS in Ray Data's retry_exceptions list
- Task IS retried on a healthy/restarted actor

The fundamental issue is Ray lacks a "fatal application error" concept:
an error where the actor should be restarted AND pending tasks retried.
The semantic gap is:
- Clean exit (exit_actor) = "I'm done" -> no retry
- Crash (os._exit) = "Something broke" -> retry

We need "crash" semantics even though this is a deliberate decision,
so os._exit(1) is the correct workaround until Ray Core adds support
for fatal application errors that trigger both restart and retry.

See: ray-project#60150
Signed-off-by: Nikhil Ghosh <nikhil@anyscale.com>
@nrghosh
Copy link
Contributor Author

nrghosh commented Jan 15, 2026

/gemini review

Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Code Review

This pull request introduces a critical fix to prevent infinite retry loops when the vLLM engine encounters a fatal error. By calling os._exit(1) upon an EngineDeadError, the Ray actor is terminated, allowing Ray's fault tolerance to restart it and correctly retry the task on a new, healthy actor. The explanation for using os._exit(1) over ray.actor.exit_actor() is thorough and well-reasoned. The code changes are concise and effective, and the corresponding test has been updated to correctly validate the new behavior by mocking os._exit. I have one suggestion to make the exception handling in the test more specific for improved robustness.

Comment on lines +811 to +812
except Exception:
pass # Code may fail after mock returns None - that's OK for this test
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

While the comment explains why an exception is expected here, catching a broad Exception can mask other potential issues in the test. Since the code path after the mocked os._exit leads to processing a None value which causes a TypeError in the StatefulStageUDF wrapper, it would be more robust to catch the specific TypeError here. This makes the test's intent clearer and less prone to hiding unrelated errors.

Suggested change
except Exception:
pass # Code may fail after mock returns None - that's OK for this test
except TypeError:
pass # Expect TypeError when processing None from the mocked exit path

@ruisearch42 ruisearch42 merged commit 3c20d6e into ray-project:master Jan 15, 2026
6 checks passed
jeffery4011 pushed a commit to jeffery4011/ray that referenced this pull request Jan 20, 2026
…ject#60145)

## Why are these changes needed?

When `EngineDeadError` occurs, the vLLM engine subprocess is dead but
the Ray actor process is still alive. Previously, we re-raised the
exception, but this causes **task retries to go to the SAME actor**
(actor methods are bound to specific instances), creating an infinite
retry loop on the broken actor.

### The Problem

```
vLLM engine subprocess crashes
       ↓
EngineDeadError raised
       ↓
Exception re-raised (actor stays ALIVE)
       ↓
Ray: actor_task_retry_on_errors triggers retry
       ↓
Retry goes to SAME actor (actor methods are bound)
       ↓
Same actor, engine still dead → EngineDeadError
       ↓
Infinite loop (with max_task_retries=-1)
```

### The Fix

Call `os._exit(1)` to exit the actor. This triggers Ray to:
1. Mark the actor as `RESTARTING`
2. Create a replacement actor with a fresh vLLM engine
3. Route task retries to healthy actors (Ray Data excludes `RESTARTING`
actors from dispatch)

This leverages Ray Data's existing fault tolerance infrastructure:
- `max_restarts=-1` (default) enables actor replacement
- `max_task_retries=-1` (default) enables task retry

### Why `os._exit(1)` instead of `ray.actor.exit_actor()`?

We must use `os._exit(1)` rather than `ray.actor.exit_actor()` because
they produce different exit types with different retry behavior:

| Exit Method | Exit Type | Exception Raised | Retried? |
|-------------|-----------|------------------|----------|
| `os._exit(1)` | `SYSTEM_ERROR` | `RaySystemError` | Yes |
| `ray.actor.exit_actor()` | `INTENDED_USER_EXIT` | `ActorDiedError` |
No |

The root cause is that Ray Data only adds `RaySystemError` to its
`retry_exceptions` list (in `_add_system_error_to_retry_exceptions()`).
Since `ActorDiedError` is NOT a subclass of `RaySystemError` (they're
siblings in the exception hierarchy), tasks that fail due to
`ray.actor.exit_actor()` are not retried.

**The semantic gap**: Ray currently lacks a "fatal application error"
concept - an error where the actor should be restarted AND pending tasks
retried. The available options are:
- Clean exit (`exit_actor`) = "I'm intentionally done" → no retry
- Crash (`os._exit`) = "Something broke unexpectedly" → retry

We need the "crash" semantics even though this is a deliberate decision,
so `os._exit(1)` is the correct workaround until Ray Core adds explicit
support for fatal application errors.

See: ray-project#60150

cc @goutamvenkat-anyscale

### Validation

We created a minimal reproduction script demonstrating:
1. **The problem**: All retries go to the same broken actor (same PID)
2. **The fix**: Actor exits → replacement created → job succeeds with
multiple PIDs

```python
# Demo output showing fix works:
[SUCCESS] Processed 20 rows!
PIDs that processed batches: {708593, 708820}
-> Multiple PIDs = replacement actor joined and processed work
```

Full reproduction:
https://gist.github.com/nrghosh/c18e514a975144a238511012774bab8b

## Related issue number

Fixes ray-project#59522

## Checks

- [x] I've signed off every commit
- [x] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
temporary file handling method, I've added it in
`doc/source/ray-core/api/doc/ray.util.temp_files.rst`.
- [ ] I've made sure the tests are passing. Note that there might be a
few flaky tests.

---------

Signed-off-by: Nikhil Ghosh <nikhil@anyscale.com>
Signed-off-by: jeffery4011 <jefferyshen1015@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

data Ray Data-related issues go add ONLY when ready to merge, run all tests llm

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[data][llm] LLM Batch Inference Resiliency Thread

4 participants