-
Notifications
You must be signed in to change notification settings - Fork 7.1k
[Data] Add preemption test for DataOpTask and refactor test utilities
#57883
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
Changes from 1 commit
cece106
91b8e46
8fbc515
e0ff24a
368a86a
41f90b6
2b87518
49387b0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -101,6 +101,12 @@ def __init__( | |
| streaming_gen: ObjectRefGenerator, | ||
| output_ready_callback: Callable[[RefBundle], None] = lambda bundle: None, | ||
| task_done_callback: Callable[[Optional[Exception]], None] = lambda exc: None, | ||
| block_ready_callback: Callable[ | ||
| [ray.ObjectRef[Block]], None | ||
| ] = lambda block_ref: None, | ||
| metadata_ready_callback: Callable[ | ||
| [ray.ObjectRef[BlockMetadata]], None | ||
| ] = lambda metadata_ref: None, | ||
| task_resource_bundle: Optional[ExecutionResources] = None, | ||
| ): | ||
| """Create a DataOpTask | ||
|
|
@@ -110,6 +116,10 @@ def __init__( | |
| output_ready_callback: The callback to call when a new RefBundle is output | ||
| from the generator. | ||
| task_done_callback: The callback to call when the task is done. | ||
| block_ready_callback: A callback that's invoked when a new block reference | ||
| is ready. This is exposed as a seam for testing. | ||
| metadata_ready_callback: A callback that's invoked when a new block metadata | ||
| reference is ready. This is exposed as a seam for testing. | ||
| task_resource_bundle: The execution resources of this task. | ||
| """ | ||
| super().__init__(task_index, task_resource_bundle) | ||
|
|
@@ -120,6 +130,8 @@ def __init__( | |
| self._streaming_gen = streaming_gen | ||
| self._output_ready_callback = output_ready_callback | ||
| self._task_done_callback = task_done_callback | ||
| self._block_ready_callback = block_ready_callback | ||
| self._metadata_ready_callback = metadata_ready_callback | ||
|
|
||
| # If the generator hasn't produced block metadata yet, or if the block metadata | ||
| # object isn't available after we get a reference, we need store the pending | ||
|
|
@@ -128,6 +140,8 @@ def __init__( | |
| self._pending_block_ref: ray.ObjectRef[Block] = ray.ObjectRef.nil() | ||
| self._pending_meta_ref: ray.ObjectRef[BlockMetadata] = ray.ObjectRef.nil() | ||
|
|
||
| self._has_finished = False | ||
|
|
||
| def get_waitable(self) -> ObjectRefGenerator: | ||
| return self._streaming_gen | ||
|
|
||
|
|
@@ -154,13 +168,16 @@ def on_data_ready(self, max_bytes_to_read: Optional[int]) -> int: | |
| ) | ||
| except StopIteration: | ||
| self._task_done_callback(None) | ||
| self._has_finished = True | ||
| break | ||
|
|
||
| if self._pending_block_ref.is_nil(): | ||
| # The generator currently doesn't have new output. | ||
| # And it's not stopped yet. | ||
| break | ||
|
|
||
| self._block_ready_callback(self._pending_block_ref) | ||
|
|
||
| if self._pending_meta_ref.is_nil(): | ||
| try: | ||
| self._pending_meta_ref = self._streaming_gen._next_sync( | ||
|
|
@@ -178,13 +195,16 @@ def on_data_ready(self, max_bytes_to_read: Optional[int]) -> int: | |
| assert False, "Above ray.get should raise an exception." | ||
| except Exception as ex: | ||
| self._task_done_callback(ex) | ||
| self._has_finished = True | ||
| raise ex from None | ||
|
|
||
| if self._pending_meta_ref.is_nil(): | ||
| # We have a reference to the block but the metadata isn't ready | ||
| # yet. | ||
| break | ||
|
|
||
| self._metadata_ready_callback(self._pending_block_ref) | ||
|
||
|
|
||
| try: | ||
| # The timeout for `ray.get` includes the time required to ship the | ||
| # block metadata to this node. So, if we set the timeout to 0, `ray.get` | ||
|
|
@@ -220,6 +240,10 @@ def on_data_ready(self, max_bytes_to_read: Optional[int]) -> int: | |
|
|
||
| return bytes_read | ||
|
|
||
| @property | ||
| def has_finished(self) -> bool: | ||
| return self._has_finished | ||
|
|
||
|
|
||
| class MetadataOpTask(OpTask): | ||
| """Represents an OpTask that only handles metadata, instead of Block data.""" | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2,7 +2,7 @@ | |
| import time | ||
| import unittest | ||
| from concurrent.futures import ThreadPoolExecutor | ||
| from typing import Optional | ||
| from typing import List, Literal, Optional, Union | ||
| from unittest.mock import MagicMock, patch | ||
|
|
||
| import numpy as np | ||
|
|
@@ -973,12 +973,34 @@ def test_create_topology_metadata_with_sub_stages(): | |
| assert sub_stage2.id.endswith("_sub_1") | ||
|
|
||
|
|
||
| class TestDataOpTask: | ||
| def test_on_data_ready_single_output(self, ray_start_regular_shared): | ||
| @ray.remote | ||
| def map_task(): | ||
| def create_stub_streaming_gen( | ||
| block_nbytes: List[int], raise_exception: Optional[Exception] = None | ||
| ) -> ray.ObjectRefGenerator: | ||
| """Creating a streaming generator for testing. | ||
|
|
||
| The streaming generator passed to the ``DataOpTask`` constructor must yield blocks | ||
| then block metadata, and buffer the number of blocks specified by | ||
| ``_max_num_blocks_in_streaming_gen_buffer``. This function is a utility to create | ||
| streaming generators that satisfy these requirements. | ||
|
|
||
| Args: | ||
| block_nbytes: A list of the sizes of blocks yielded by the returned streaming | ||
| generator. | ||
| raise_exception: An exception that the streaming generator immediately raises. | ||
|
|
||
| Returns: | ||
| A streaming generator that you can pass to ``DataOpTask``. | ||
| """ | ||
|
|
||
| @ray.remote | ||
| def stub_map_task(): | ||
| if raise_exception is not None: | ||
| raise raise_exception | ||
|
|
||
| for nbytes in block_nbytes: | ||
| # Create a block with a single row of the specified size. | ||
| builder = DelegatingBlockBuilder() | ||
| builder.add_batch({"data": np.zeros((1, 128 * MiB), dtype=np.uint8)}) | ||
| builder.add_batch({"data": np.zeros((1, nbytes), dtype=np.uint8)}) | ||
| block = builder.build() | ||
| yield block | ||
|
|
||
|
|
@@ -988,68 +1010,120 @@ def map_task(): | |
| block_metadata, schema=block_accessor.schema() | ||
| ) | ||
|
|
||
| generator_backpressure_num_objects = ( | ||
| ray.data.DataContext.get_current()._max_num_blocks_in_streaming_gen_buffer | ||
| * 2 # Multiply by two because we yield a metadata object for each block. | ||
| ) | ||
| streaming_gen = stub_map_task.options( | ||
| _generator_backpressure_num_objects=generator_backpressure_num_objects | ||
| ).remote() | ||
|
|
||
| return streaming_gen | ||
|
|
||
|
|
||
| class TestDataOpTask: | ||
| def test_on_data_ready_single_output(self, ray_start_regular_shared): | ||
| streaming_gen = create_stub_streaming_gen(block_nbytes=[128 * MiB]) | ||
|
|
||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| def verify_output(bundle): | ||
| assert bundle.num_rows() == 1, bundle.num_rows() | ||
| assert bundle.size_bytes() == pytest.approx(128 * MiB), bundle.size_bytes() | ||
|
|
||
| has_completed = False | ||
| data_op_task = DataOpTask(0, streaming_gen, output_ready_callback=verify_output) | ||
|
|
||
| def verify_exception(exc: Optional[Exception]): | ||
| nonlocal has_completed | ||
| bytes_read = 0 | ||
| while not data_op_task.has_finished: | ||
| ray.wait([streaming_gen], fetch_local=False) | ||
| nbytes_read = data_op_task.on_data_ready(None) | ||
| bytes_read += nbytes_read | ||
|
|
||
| assert exc is None | ||
| has_completed = True | ||
| assert bytes_read == pytest.approx(128 * MiB) | ||
|
|
||
| generator_backpressure_num_objects = ( | ||
| ray.data.DataContext.get_current()._max_num_blocks_in_streaming_gen_buffer | ||
| * 2 # Multiply by two because we yield a metadata object for each block. | ||
| ) | ||
| streaming_gen = map_task.options( | ||
| _generator_backpressure_num_objects=generator_backpressure_num_objects | ||
| ).remote() | ||
| def test_on_data_ready_multiple_outputs(self, ray_start_regular_shared): | ||
| streaming_gen = create_stub_streaming_gen(block_nbytes=[128 * MiB, 128 * MiB]) | ||
|
|
||
| data_op_task = DataOpTask( | ||
| 0, | ||
| streaming_gen, | ||
| output_ready_callback=verify_output, | ||
| task_done_callback=verify_exception, | ||
| ) | ||
| def verify_output(bundle): | ||
| assert bundle.size_bytes() == pytest.approx(128 * MiB), bundle.size_bytes() | ||
|
|
||
| data_op_task = DataOpTask(0, streaming_gen, output_ready_callback=verify_output) | ||
|
|
||
| bytes_read = 0 | ||
| while not has_completed: | ||
| while not data_op_task.has_finished: | ||
| ray.wait([streaming_gen], fetch_local=False) | ||
| bytes_read += data_op_task.on_data_ready(None) | ||
| nbytes_read = data_op_task.on_data_ready(None) | ||
| bytes_read += nbytes_read | ||
|
|
||
| assert bytes_read == pytest.approx(128 * MiB) | ||
| assert bytes_read == pytest.approx(256 * MiB) | ||
|
|
||
| def test_on_data_ready_exception(self, ray_start_regular_shared): | ||
| @ray.remote | ||
| def map_task(): | ||
| assert False, "Block generation failed" | ||
| yield | ||
| streaming_gen = create_stub_streaming_gen( | ||
| block_nbytes=[128 * MiB], | ||
| raise_exception=AssertionError("Block generation failed"), | ||
| ) | ||
|
|
||
| def verify_exception(exc: Optional[Exception]): | ||
| assert isinstance(exc, AssertionError) | ||
|
|
||
| generator_backpressure_num_objects = ( | ||
| ray.data.DataContext.get_current()._max_num_blocks_in_streaming_gen_buffer | ||
| * 2 # Multiply by two because we yield a metadata object for each block. | ||
| ) | ||
| streaming_gen = map_task.options( | ||
| _generator_backpressure_num_objects=generator_backpressure_num_objects | ||
| ).remote() | ||
|
|
||
| data_op_task = DataOpTask( | ||
| 0, | ||
| streaming_gen, | ||
| task_done_callback=verify_exception, | ||
| ) | ||
|
|
||
| with pytest.raises(AssertionError, match="Block generation failed"): | ||
| while True: | ||
| while not data_op_task.has_finished: | ||
| ray.wait([streaming_gen], fetch_local=False) | ||
| data_op_task.on_data_ready(None) | ||
|
|
||
| @pytest.mark.parametrize( | ||
| "preempt_on", ["block_ready_callback", "metadata_ready_callback"] | ||
| ) | ||
| def test_on_data_ready_with_preemption( | ||
| self, | ||
| preempt_on: Union[ | ||
| Literal["block_ready_callback"], Literal["metadata_ready_callback"] | ||
| ], | ||
| ray_start_cluster_enabled, | ||
| monkeypatch, | ||
| ): | ||
| """Test that ``on_data_ready`` works when a node dies during its execution.""" | ||
|
|
||
| # 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) | ||
|
|
||
| cluster = ray_start_cluster_enabled | ||
| head_node = cluster.add_node(num_cpus=0) # noqa: F841 | ||
| worker_node = cluster.add_node(num_cpus=1) | ||
| cluster.wait_for_nodes() | ||
|
|
||
| streaming_gen = create_stub_streaming_gen(block_nbytes=[128 * MiB]) | ||
|
|
||
| def remove_and_add_back_worker_node(_): | ||
| cluster.remove_node(worker_node) | ||
|
|
||
| new_worker_node = cluster.add_node(num_cpus=1) | ||
| cluster.add_node(new_worker_node) | ||
bveeramani marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
bveeramani marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| data_op_task = DataOpTask( | ||
| 0, streaming_gen, **{preempt_on: remove_and_add_back_worker_node} | ||
| ) | ||
|
|
||
| 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) | ||
bveeramani marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| assert bytes_read == pytest.approx(128 * MiB) | ||
|
|
||
|
|
||
| if __name__ == "__main__": | ||
| import sys | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.