Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
78 changes: 63 additions & 15 deletions python/ray/data/tests/test_streaming_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -1021,6 +1021,22 @@ def stub_map_task():
return streaming_gen


@pytest.fixture
def ensure_block_metadata_stored_in_plasma(monkeypatch):
# Ray inlines small objects (including metadata) by storing them directly with
# the object reference itself rather than in the remote node's object store.
# Consequently, when the streaming executor calls `ray.get` on metadata from a
# node that has died, the call succeeds because the inlined metadata is not
# stored in the failed node's object store. To explicitly test the case where
# metadata resides in the object store (and becomes unavailable when the node
# dies), we disable inlining by setting the maximum inline size to 0. This
# simulates scenarios where metadata is too large to inline, which can occur in
# practice when schemas contain many fields.
#
# For context, see https://github.com/ray-project/ray/pull/56451.
monkeypatch.setenv("RAY_max_direct_call_object_size", 0)


class TestDataOpTask:
def test_on_data_ready_single_output(self, ray_start_regular_shared):
streaming_gen = create_stub_streaming_gen(block_nbytes=[128 * MiB])
Expand Down Expand Up @@ -1077,29 +1093,19 @@ def verify_exception(exc: Optional[Exception]):
@pytest.mark.parametrize(
"preempt_on", ["block_ready_callback", "metadata_ready_callback"]
)
def test_on_data_ready_with_preemption(
def test_on_data_ready_with_preemption_during_call(
self,
preempt_on: Union[
Literal["block_ready_callback"], Literal["metadata_ready_callback"]
],
ray_start_cluster_enabled,
monkeypatch,
ensure_block_metadata_stored_in_plasma,
):
"""Test that ``on_data_ready`` works when a node dies during its execution."""
# Shutdown Ray incase it's already initialized.
ray.shutdown()

# Ray inlines small objects (including metadata) by storing them directly with
# the object reference itself rather than in the remote node's object store.
# Consequently, when the streaming executor calls `ray.get` on metadata from a
# node that has died, the call succeeds because the inlined metadata is not
# stored in the failed node's object store. To explicitly test the case where
# metadata resides in the object store (and becomes unavailable when the node
# dies), we disable inlining by setting the maximum inline size to 0. This
# simulates scenarios where metadata is too large to inline, which can occur in
# practice when schemas contain many fields.
#
# For context, see https://github.com/ray-project/ray/pull/56451.
monkeypatch.setenv("RAY_max_direct_call_object_size", 0)

# Create a single-worker-node cluster with 1 logical CPU.
cluster = ray_start_cluster_enabled
head_node = cluster.add_node(num_cpus=0) # noqa: F841
cluster.wait_for_nodes()
Expand All @@ -1108,6 +1114,8 @@ def test_on_data_ready_with_preemption(
worker_node = cluster.add_node(num_cpus=1)
cluster.wait_for_nodes()

# Create a streaming generator that produces a single 128 MiB output block, and
# configure it so that it preempts the worker node in the specified callback.
streaming_gen = create_stub_streaming_gen(block_nbytes=[128 * MiB])

def remove_and_add_back_worker_node(_):
Expand All @@ -1120,11 +1128,51 @@ def remove_and_add_back_worker_node(_):
0, streaming_gen, **{preempt_on: remove_and_add_back_worker_node}
)

# Run the task to completion.
bytes_read = 0
while not data_op_task.has_finished:
ray.wait([streaming_gen], fetch_local=False)
bytes_read += data_op_task.on_data_ready(None)

# Ensure that we read the expected amount of data. Since the streaming generator
# yields a single 128 MiB block, we should read 128 MiB.
assert bytes_read == pytest.approx(128 * MiB)

def test_on_data_ready_with_preemption_after_wait(
self, ray_start_cluster_enabled, ensure_block_metadata_stored_in_plasma
):
# Shutdown Ray incase it's already initialized.
ray.shutdown()

# Create a single-worker-node cluster with 1 logical CPU.
cluster = ray_start_cluster_enabled
head_node = cluster.add_node(num_cpus=0) # noqa: F841
cluster.wait_for_nodes()
ray.init()

worker_node = cluster.add_node(num_cpus=1)
cluster.wait_for_nodes()

# Create a streaming generator that produces a single 128 MiB output block.
streaming_gen = create_stub_streaming_gen(block_nbytes=[128 * MiB])
data_op_task = DataOpTask(0, streaming_gen)

# Wait for the block to be ready, then remove the worker node.
ray.wait([streaming_gen], fetch_local=False)
cluster.remove_node(worker_node)

# The block shouldn't be available anymore, so we shouldn't read any data.
bytes_read = data_op_task.on_data_ready(None)
assert bytes_read == 0

# Re-add the worker node, and run the task to completion.
new_worker_node = cluster.add_node(num_cpus=1) # noqa: F841
cluster.wait_for_nodes()
while not data_op_task.has_finished:
ray.wait([streaming_gen], fetch_local=False)
bytes_read += data_op_task.on_data_ready(None)

# We should now be able to read the 128 MiB block.
assert bytes_read == pytest.approx(128 * MiB)


Expand Down
3 changes: 3 additions & 0 deletions python/ray/tests/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -535,6 +535,9 @@ def class_ray_instance():

@contextmanager
def _ray_start(**kwargs):
if ray.is_initialized():
ray.shutdown()

init_kwargs = get_default_fixture_ray_kwargs()
init_kwargs.update(kwargs)
# Start the Ray processes.
Expand Down