-
Notifications
You must be signed in to change notification settings - Fork 7k
Open
Labels
bugSomething that is supposed to be working; but isn'tSomething that is supposed to be working; but isn'tdataRay Data-related issuesRay Data-related issuesstabilitytriageNeeds triage (eg: priority, bug/not-bug, and owning component)Needs triage (eg: priority, bug/not-bug, and owning component)
Description
What happened + What you expected to happen
What happened:
When a Ray Data map_batches actor is OOM killed mid-task, the entire pipeline hangs indefinitely. The symptoms are:
- The map_batches task fails and Ray OOM kills the actor
- The task appears to keep "running" and takes much longer than average (hung state)
- All nodes stop making progress - no tasks complete
- The pipeline never recovers, even with actor restart policies enabled
Root Cause:
When DataOpTask.on_data_ready() raises an exception (e.g., RayActorError from a dead actor), the exception is caught in process_completed_tasks()
at streaming_executor_state.py:598, but task._task_done_callback() is never called. This means:
- on_task_completed() is never called on the actor pool
- num_tasks_in_flight counter is never decremented
- The task remains in _data_tasks dict forever
- should_add_input() returns False (thinks actor slots are still occupied)
- No new tasks can be scheduled → deadlock
What I expected:
When an actor dies mid-task, the task cleanup callback chain should still execute to:
- Decrement num_tasks_in_flight
- Remove the task from _data_tasks
- Free the actor pool slot so new tasks can be scheduled
- Allow the pipeline to continue (with actor restart or error propagation)
Versions / Dependencies
- Ray version: 2.52.1 (also affects master branch)
- Python version: 3.12
- OS: macOS / Linux
Reproduction script
import time
import ray
import numpy as np
ray.init()
def memory_hungry_fn(batch):
"""Allocate memory to trigger OOM."""
# Allocate ~500MB per batch to stress memory
_ = np.zeros((500, 1024, 1024), dtype=np.uint8)
time.sleep(0.1)
return batch
# Create dataset with enough data to keep actors busy
ds = ray.data.range(1000)
# Use actor pool with small memory allocation to trigger OOM
ds = ds.map_batches(
memory_hungry_fn,
batch_size=10,
concurrency=4, # Multiple actors
)
# This will hang after actor OOM kill
try:
for batch in ds.iter_batches(batch_size=100):
print(f"Got batch with {len(batch['id'])} rows")
except Exception as e:
print(f"Error: {e}")
print("Done") # Never reached if bug is present
Alternative minimal reproduction (unit test level):
import time
import ray
from ray.data._internal.compute import ActorPoolStrategy
from ray.data._internal.execution.operators.map_operator import MapOperator
from ray.data._internal.execution.operators.input_data_buffer import InputDataBuffer
from ray.data._internal.execution.operators.map_transformer import (
BlockMapTransformFn, MapTransformer,
)
from ray.data._internal.execution.streaming_executor_state import (
build_streaming_topology, process_completed_tasks,
)
from ray.data._internal.execution.interfaces import ExecutionOptions
from ray.data._internal.execution.util import make_ref_bundles
from ray.data.context import DataContext
ray.init()
@ray.remote(num_cpus=0)
class BlockingSignal:
def wait_forever(self):
time.sleep(999999)
signal = BlockingSignal.remote()
ctx = DataContext.get_current()
def block_forever_then_passthrough(block_iter):
ray.get(signal.wait_forever.remote()) # Block forever
yield from block_iter
transformer = MapTransformer([BlockMapTransformFn(block_forever_then_passthrough)])
input_op = InputDataBuffer(ctx, make_ref_bundles([[0]]))
op = MapOperator.create(
transformer,
input_op=input_op,
data_context=ctx,
name="TestActorDiesCleanup",
compute_strategy=ActorPoolStrategy(size=1),
ray_remote_args={"max_restarts": 0},
)
topology = build_streaming_topology(op, ExecutionOptions())
# Wait for actor to be ready
while not op.should_add_input():
process_completed_tasks(topology, [], max_errored_blocks=-1)
# Submit task
op.add_input(input_op.get_next(), 0)
print(f"Before kill: num_tasks_in_flight={op._actor_pool.num_tasks_in_flight()}")
# Kill actor mid-task (simulates OOM kill)
actor = next(iter(op._actor_pool.running_actors().keys()))
ray.kill(actor, no_restart=True)
# Process the failure
deadline = time.time() + 10
while True:
process_completed_tasks(topology, [], max_errored_blocks=-1)
tasks_in_flight = op._actor_pool.num_tasks_in_flight()
print(f"After kill: num_tasks_in_flight={tasks_in_flight}")
if tasks_in_flight == 0:
print("SUCCESS: Task cleanup happened correctly!")
break
if time.time() > deadline:
print("BUG: num_tasks_in_flight stuck at", tasks_in_flight)
print("This causes pipeline hang - no new tasks can be scheduled")
break
time.sleep(0.5)
Issue Severity
None
Metadata
Metadata
Assignees
Labels
bugSomething that is supposed to be working; but isn'tSomething that is supposed to be working; but isn'tdataRay Data-related issuesRay Data-related issuesstabilitytriageNeeds triage (eg: priority, bug/not-bug, and owning component)Needs triage (eg: priority, bug/not-bug, and owning component)