From 245ff6e1d4a8b8c3531d0214fc08b938719c94a2 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Mon, 29 Jul 2024 23:17:13 +0000 Subject: [PATCH 001/111] execute schedule --- python/ray/dag/compiled_dag_node.py | 137 ++++++++++++------ .../experimental/test_accelerated_dag.py | 26 ++-- .../channel/serialization_context.py | 21 +++ 3 files changed, 126 insertions(+), 58 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 0c7f8cff58eff..ba523465f1eb6 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -25,6 +25,7 @@ SynchronousWriter, AwaitableBackgroundReader, AwaitableBackgroundWriter, + ChannelContext, ) from ray.util.annotations import DeveloperAPI @@ -38,6 +39,26 @@ ) from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy +from enum import Enum, auto + + +class DAGNodeOperationType(Enum): + READ = auto() + COMPUTE = auto() + WRITE = auto() + + +class DAGNodeOperation: + def __init__( + self, + bind_index: int, + operation_type: DAGNodeOperationType, + ): + self.bind_index = bind_index + self.type = operation_type + + def __repr__(self) -> str: + return f"DAGNodeOperation({self.bind_index}, {self.type})" # Holds the input arguments for an accelerated DAG node. @@ -84,6 +105,7 @@ def do_allocate_channel( def do_exec_tasks( self, tasks: List["ExecutableTask"], + schedule: List[Tuple[int, DAGNodeOperation]], ) -> None: """Generic actor method to begin executing the tasks belonging to an actor. This runs an infinite loop to run each task in turn (following the order specified @@ -103,11 +125,12 @@ def do_exec_tasks( while True: if done: break - for idx, task in enumerate(tasks): - done = _exec_task(self, task, idx) + for idx, operation in schedule: + operation_type = operation.type + task = tasks[idx] + done = _exec_task(self, task, idx, operation_type) if done: break - except Exception: logging.exception("Compiled DAG task exited with exception") raise @@ -150,7 +173,9 @@ def _wrap_exception(exc): return wrapped -def _exec_task(self, task: "ExecutableTask", idx: int) -> bool: +def _exec_task( + self, task: "ExecutableTask", idx: int, op_type: DAGNodeOperationType +) -> bool: """ Execute the task. Args: @@ -162,41 +187,47 @@ def _exec_task(self, task: "ExecutableTask", idx: int) -> bool: # TODO: for cases where output is passed as input to a task on # the same actor, introduce a "IntraProcessChannel" to avoid the overhead # of serialization/deserialization and synchronization. - method = getattr(self, task.method_name) input_reader = self._input_readers[idx] output_writer = self._output_writers[idx] - res = None - try: - res = input_reader.read() - except RayChannelError: - # Channel closed. Exit the loop. - return True - - try: - _process_return_vals(res, return_single_output=False) - except Exception as exc: - # Previous task raised an application-level exception. - # Propagate it and skip the actual task. We don't need to wrap the - # exception in a RayTaskError here because it has already been wrapped - # by the previous task. - output_writer.write(exc) - return False - - resolved_inputs = [] - for task_input in task.task_inputs: - resolved_inputs.append(task_input.resolve(res)) - - try: - output_val = method(*resolved_inputs, **task.resolved_kwargs) - except Exception as exc: - output_val = _wrap_exception(exc) - - try: - output_writer.write(output_val) - except RayChannelError: - # Channel closed. Exit the loop. - return True - + ctx = ChannelContext.get_current().serialization_context + + if op_type == DAGNodeOperationType.READ: + try: + res = input_reader.read() + ctx.set_intermediate_result(idx, op_type, res) + except RayChannelError: + # Channel closed. Exit the loop. + return True + elif op_type == DAGNodeOperationType.COMPUTE: + res = ctx.get_intermediate_result(idx, DAGNodeOperationType.READ) + method = getattr(self, task.method_name) + try: + _process_return_vals(res, return_single_output=False) + except Exception as exc: + # Previous task raised an application-level exception. + # Propagate it and skip the actual task. We don't need to wrap the + # exception in a RayTaskError here because it has already been wrapped + # by the previous task. + ctx.set_intermediate_result(idx, op_type, exc) + return False + + resolved_inputs = [] + for task_input in task.task_inputs: + resolved_inputs.append(task_input.resolve(res)) + + try: + output_val = method(*resolved_inputs, **task.resolved_kwargs) + # TODO: Cache in buffer. + except Exception as exc: + output_val = _wrap_exception(exc) + ctx.set_intermediate_result(idx, op_type, output_val) + elif op_type == DAGNodeOperationType.WRITE: + output_val = ctx.get_intermediate_result(idx, DAGNodeOperationType.COMPUTE) + try: + output_writer.write(output_val) + except RayChannelError: + # Channel closed. Exit the loop. + return True return False @@ -990,7 +1021,6 @@ def _get_or_compile( # Create executable tasks for each actor for actor_handle, tasks in self.actor_to_tasks.items(): executable_tasks = [] - worker_fn = None for task in tasks: resolved_args = [] has_at_least_one_channel_input = False @@ -1024,19 +1054,19 @@ def _get_or_compile( task.kwargs, ) executable_tasks.append(executable_task) - if worker_fn is None: - worker_fn = task.dag_node._get_remote_method("__ray_call__") # Sort executable tasks based on their bind index, i.e., submission order # so that they will be executed in that order. executable_tasks.sort(key=lambda task: task.bind_index) - self.actor_to_executable_tasks[actor_handle] = executable_tasks - # Assign the task with the correct input and output buffers. - self.worker_task_refs[ - task.dag_node._get_actor_handle() - ] = worker_fn.options(concurrency_group="_ray_system").remote( + + # Build schedule for each actor + for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): + self.worker_task_refs[actor_handle] = actor_handle.__ray_call__.options( + concurrency_group="_ray_system" + ).remote( do_exec_tasks, executable_tasks, + self._build_execution_schedule(actor_handle), ) self.dag_output_channels = [] @@ -1075,6 +1105,23 @@ def _get_or_compile( self._dag_submitter.start() self._dag_output_fetcher.start() + def _build_execution_schedule( + self, actor_handle: "ray.actor.ActorHandle" + ) -> List[Tuple[int, DAGNodeOperation]]: + schedule = [] + for idx, task in enumerate(self.actor_to_executable_tasks[actor_handle]): + bind_index = task.bind_index + schedule.append( + (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.READ)) + ) + schedule.append( + (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.COMPUTE)) + ) + schedule.append( + (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.WRITE)) + ) + return schedule + def _detect_deadlock(self) -> bool: """ Create a graph with the following 3 rules, and then use diff --git a/python/ray/dag/tests/experimental/test_accelerated_dag.py b/python/ray/dag/tests/experimental/test_accelerated_dag.py index 859ea7c7ab0cf..5707956689e51 100644 --- a/python/ray/dag/tests/experimental/test_accelerated_dag.py +++ b/python/ray/dag/tests/experimental/test_accelerated_dag.py @@ -788,7 +788,7 @@ def test_compiled_dag_ref_del(ray_start_regular): compiled_dag.teardown() -def test_dag_fault_tolerance_chain(ray_start_regular_shared): +def test_dag_fault_tolerance_chain(ray_start_regular): actors = [ Actor.remote(0, fail_after=100 if i == 0 else None, sys_exit=False) for i in range(4) @@ -831,7 +831,7 @@ def test_dag_fault_tolerance_chain(ray_start_regular_shared): compiled_dag.teardown() -def test_dag_fault_tolerance(ray_start_regular_shared): +def test_dag_fault_tolerance(ray_start_regular): actors = [ Actor.remote(0, fail_after=100 if i == 0 else None, sys_exit=False) for i in range(4) @@ -872,7 +872,7 @@ def test_dag_fault_tolerance(ray_start_regular_shared): compiled_dag.teardown() -def test_dag_fault_tolerance_sys_exit(ray_start_regular_shared): +def test_dag_fault_tolerance_sys_exit(ray_start_regular): actors = [ Actor.remote(0, fail_after=100 if i == 0 else None, sys_exit=True) for i in range(4) @@ -912,7 +912,7 @@ def test_dag_fault_tolerance_sys_exit(ray_start_regular_shared): compiled_dag.teardown() -def test_dag_teardown_while_running(ray_start_regular_shared): +def test_dag_teardown_while_running(ray_start_regular): a = Actor.remote(0) with InputNode() as inp: @@ -939,7 +939,7 @@ def test_dag_teardown_while_running(ray_start_regular_shared): @pytest.mark.parametrize("max_queue_size", [None, 2]) -def test_asyncio(ray_start_regular_shared, max_queue_size): +def test_asyncio(ray_start_regular, max_queue_size): a = Actor.remote(0) with InputNode() as i: dag = a.echo.bind(i) @@ -965,7 +965,7 @@ async def main(i): @pytest.mark.parametrize("max_queue_size", [None, 2]) -def test_asyncio_exceptions(ray_start_regular_shared, max_queue_size): +def test_asyncio_exceptions(ray_start_regular, max_queue_size): a = Actor.remote(0) with InputNode() as i: dag = a.inc.bind(i) @@ -1005,7 +1005,7 @@ async def main(): class TestCompositeChannel: - def test_composite_channel_one_actor(self, ray_start_regular_shared): + def test_composite_channel_one_actor(self, ray_start_regular): """ In this test, there are three 'inc' tasks on the same Ray actor, chained together. Therefore, the DAG will look like this: @@ -1039,7 +1039,7 @@ def test_composite_channel_one_actor(self, ray_start_regular_shared): compiled_dag.teardown() - def test_composite_channel_two_actors(self, ray_start_regular_shared): + def test_composite_channel_two_actors(self, ray_start_regular): """ In this test, there are three 'inc' tasks on the two Ray actors, chained together. Therefore, the DAG will look like this: @@ -1073,7 +1073,7 @@ def test_composite_channel_two_actors(self, ray_start_regular_shared): compiled_dag.teardown() - def test_composite_channel_multi_output(self, ray_start_regular_shared): + def test_composite_channel_multi_output(self, ray_start_regular): """ Driver -> a.inc -> a.inc ---> Driver | | @@ -1100,7 +1100,7 @@ def test_composite_channel_multi_output(self, ray_start_regular_shared): compiled_dag.teardown() - def test_intra_process_channel_with_multi_readers(self, ray_start_regular_shared): + def test_intra_process_channel_with_multi_readers(self, ray_start_regular): """ In this test, there are three 'echo' tasks on the same Ray actor. The DAG will look like this: @@ -1134,7 +1134,7 @@ def test_intra_process_channel_with_multi_readers(self, ray_start_regular_shared compiled_dag.teardown() -def test_simulate_pipeline_parallelism(ray_start_regular_shared): +def test_simulate_pipeline_parallelism(ray_start_regular): """ This pattern simulates the case of pipeline parallelism training, where `w0_input` reads data from the driver, and the fan-out tasks, `d00`, `d01`, and `d02`, use @@ -1209,7 +1209,7 @@ def read_input(self, input): output_dag.teardown() -def test_channel_read_after_close(ray_start_regular_shared): +def test_channel_read_after_close(ray_start_regular): # Tests that read to a channel after accelerated DAG teardown raises a # RayChannelError exception as the channel is closed (see issue #46284). @ray.remote @@ -1229,7 +1229,7 @@ def foo(self, arg): ray.get(ref) -def test_channel_write_after_close(ray_start_regular_shared): +def test_channel_write_after_close(ray_start_regular): # Tests that write to a channel after accelerated DAG teardown raises a # RayChannelError exception as the channel is closed. @ray.remote diff --git a/python/ray/experimental/channel/serialization_context.py b/python/ray/experimental/channel/serialization_context.py index 92599a0f8ee86..999460e259966 100644 --- a/python/ray/experimental/channel/serialization_context.py +++ b/python/ray/experimental/channel/serialization_context.py @@ -18,10 +18,31 @@ def __init__(self): # The number of readers for each channel. When the number of readers # reaches 0, remove the data from the buffer. self.channel_id_to_num_readers: Dict[str, int] = {} + self.intermediate_results_buffer: Dict[ + int, Dict["DAGNodeOperationType", Any] + ] = {} def set_use_external_transport(self, use_external_transport: bool) -> None: self.use_external_transport = use_external_transport + def set_intermediate_result( + self, bind_index: int, op_type: "DAGNodeOperationType", value: Any + ) -> None: + if bind_index not in self.intermediate_results_buffer: + self.intermediate_results_buffer[bind_index] = {} + self.intermediate_results_buffer[bind_index][op_type] = value + + def get_intermediate_result( + self, bind_index: int, op_type: "DAGNodeOperationType" + ) -> Any: + assert ( + bind_index in self.intermediate_results_buffer + ), f"Bind index {bind_index} does not exist in the buffer." + assert ( + op_type in self.intermediate_results_buffer[bind_index] + ), f"Operation type {op_type} does not exist in the buffer." + return self.intermediate_results_buffer[bind_index].pop(op_type) + def set_data(self, channel_id: str, value: Any, num_readers: int) -> None: assert num_readers > 0, "num_readers must be greater than 0." assert ( From 52453e2d5438e165faffa66965ff9031af3b1f1c Mon Sep 17 00:00:00 2001 From: kaihsun Date: Mon, 29 Jul 2024 23:51:25 +0000 Subject: [PATCH 002/111] change operation type enum from int to str --- python/ray/dag/compiled_dag_node.py | 8 ++++---- .../ray/experimental/channel/serialization_context.py | 10 +++------- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index ba523465f1eb6..a39b6afcb623f 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -39,13 +39,13 @@ ) from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy -from enum import Enum, auto +from enum import Enum class DAGNodeOperationType(Enum): - READ = auto() - COMPUTE = auto() - WRITE = auto() + READ = "READ" + COMPUTE = "COMPUTE" + WRITE = "WRITE" class DAGNodeOperation: diff --git a/python/ray/experimental/channel/serialization_context.py b/python/ray/experimental/channel/serialization_context.py index 999460e259966..c51d104fd4660 100644 --- a/python/ray/experimental/channel/serialization_context.py +++ b/python/ray/experimental/channel/serialization_context.py @@ -18,23 +18,19 @@ def __init__(self): # The number of readers for each channel. When the number of readers # reaches 0, remove the data from the buffer. self.channel_id_to_num_readers: Dict[str, int] = {} - self.intermediate_results_buffer: Dict[ - int, Dict["DAGNodeOperationType", Any] - ] = {} + self.intermediate_results_buffer: Dict[int, Dict[str, Any]] = {} def set_use_external_transport(self, use_external_transport: bool) -> None: self.use_external_transport = use_external_transport def set_intermediate_result( - self, bind_index: int, op_type: "DAGNodeOperationType", value: Any + self, bind_index: int, op_type: str, value: Any ) -> None: if bind_index not in self.intermediate_results_buffer: self.intermediate_results_buffer[bind_index] = {} self.intermediate_results_buffer[bind_index][op_type] = value - def get_intermediate_result( - self, bind_index: int, op_type: "DAGNodeOperationType" - ) -> Any: + def get_intermediate_result(self, bind_index: int, op_type: str) -> Any: assert ( bind_index in self.intermediate_results_buffer ), f"Bind index {bind_index} does not exist in the buffer." From f7ea8779cbcbb78c4a277d638fb8e963155e9c95 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Mon, 29 Jul 2024 23:58:11 +0000 Subject: [PATCH 003/111] pass str as dict key --- python/ray/dag/compiled_dag_node.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index a39b6afcb623f..32336a4e3b19b 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -194,12 +194,12 @@ def _exec_task( if op_type == DAGNodeOperationType.READ: try: res = input_reader.read() - ctx.set_intermediate_result(idx, op_type, res) + ctx.set_intermediate_result(idx, op_type.value, res) except RayChannelError: # Channel closed. Exit the loop. return True elif op_type == DAGNodeOperationType.COMPUTE: - res = ctx.get_intermediate_result(idx, DAGNodeOperationType.READ) + res = ctx.get_intermediate_result(idx, DAGNodeOperationType.READ.value) method = getattr(self, task.method_name) try: _process_return_vals(res, return_single_output=False) @@ -208,7 +208,7 @@ def _exec_task( # Propagate it and skip the actual task. We don't need to wrap the # exception in a RayTaskError here because it has already been wrapped # by the previous task. - ctx.set_intermediate_result(idx, op_type, exc) + ctx.set_intermediate_result(idx, op_type.value, exc) return False resolved_inputs = [] @@ -220,9 +220,11 @@ def _exec_task( # TODO: Cache in buffer. except Exception as exc: output_val = _wrap_exception(exc) - ctx.set_intermediate_result(idx, op_type, output_val) + ctx.set_intermediate_result(idx, op_type.value, output_val) elif op_type == DAGNodeOperationType.WRITE: - output_val = ctx.get_intermediate_result(idx, DAGNodeOperationType.COMPUTE) + output_val = ctx.get_intermediate_result( + idx, DAGNodeOperationType.COMPUTE.value + ) try: output_writer.write(output_val) except RayChannelError: From 9e411c72d8dae3514b6408a640d7e9b6d1585e91 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Tue, 30 Jul 2024 00:14:33 +0000 Subject: [PATCH 004/111] actor_to_execution_schedule --- python/ray/dag/compiled_dag_node.py | 39 ++++++++++++++++------------- 1 file changed, 22 insertions(+), 17 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 32336a4e3b19b..50c7e81ae1366 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -555,6 +555,9 @@ def __init__( self.actor_to_executable_tasks: Dict[ "ray.actor.ActorHandle", List["ExecutableTask"] ] = {} + self.actor_to_execution_schedule: Dict[ + "ray.actor.ActorHandle", List[DAGNodeOperation] + ] = defaultdict(list) # Mapping from the actor handle to the node ID that the actor is on. self.actor_to_node_id: Dict["ray.actor.ActorHandle", str] = {} @@ -1062,13 +1065,16 @@ def _get_or_compile( self.actor_to_executable_tasks[actor_handle] = executable_tasks # Build schedule for each actor + self._build_execution_schedule() + + # Execute schedule for each actor for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): self.worker_task_refs[actor_handle] = actor_handle.__ray_call__.options( concurrency_group="_ray_system" ).remote( do_exec_tasks, executable_tasks, - self._build_execution_schedule(actor_handle), + self.actor_to_execution_schedule[actor_handle], ) self.dag_output_channels = [] @@ -1107,22 +1113,21 @@ def _get_or_compile( self._dag_submitter.start() self._dag_output_fetcher.start() - def _build_execution_schedule( - self, actor_handle: "ray.actor.ActorHandle" - ) -> List[Tuple[int, DAGNodeOperation]]: - schedule = [] - for idx, task in enumerate(self.actor_to_executable_tasks[actor_handle]): - bind_index = task.bind_index - schedule.append( - (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.READ)) - ) - schedule.append( - (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.COMPUTE)) - ) - schedule.append( - (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.WRITE)) - ) - return schedule + def _build_execution_schedule(self): + for actor_handle in self.actor_to_executable_tasks: + schedule = [] + for idx, task in enumerate(self.actor_to_executable_tasks[actor_handle]): + bind_index = task.bind_index + schedule.append( + (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.READ)) + ) + schedule.append( + (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.COMPUTE)) + ) + schedule.append( + (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.WRITE)) + ) + self.actor_to_execution_schedule[actor_handle] = schedule def _detect_deadlock(self) -> bool: """ From 77e3a2610c896d87daf2cf7ba061de32e7a1072f Mon Sep 17 00:00:00 2001 From: kaihsun Date: Tue, 30 Jul 2024 00:43:11 +0000 Subject: [PATCH 005/111] remove bind_index from DAGNodeOperation --- python/ray/dag/compiled_dag_node.py | 27 +++++++++++---------------- 1 file changed, 11 insertions(+), 16 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 50c7e81ae1366..56ab28aeec298 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -51,14 +51,15 @@ class DAGNodeOperationType(Enum): class DAGNodeOperation: def __init__( self, - bind_index: int, + idx: int, operation_type: DAGNodeOperationType, ): - self.bind_index = bind_index + # not bind_index + self.idx = idx self.type = operation_type def __repr__(self) -> str: - return f"DAGNodeOperation({self.bind_index}, {self.type})" + return f"DAGNodeOperation({self.idx}, {self.type})" # Holds the input arguments for an accelerated DAG node. @@ -105,7 +106,7 @@ def do_allocate_channel( def do_exec_tasks( self, tasks: List["ExecutableTask"], - schedule: List[Tuple[int, DAGNodeOperation]], + schedule: List[DAGNodeOperation], ) -> None: """Generic actor method to begin executing the tasks belonging to an actor. This runs an infinite loop to run each task in turn (following the order specified @@ -125,8 +126,9 @@ def do_exec_tasks( while True: if done: break - for idx, operation in schedule: + for operation in schedule: operation_type = operation.type + idx = operation.idx task = tasks[idx] done = _exec_task(self, task, idx, operation_type) if done: @@ -1116,17 +1118,10 @@ def _get_or_compile( def _build_execution_schedule(self): for actor_handle in self.actor_to_executable_tasks: schedule = [] - for idx, task in enumerate(self.actor_to_executable_tasks[actor_handle]): - bind_index = task.bind_index - schedule.append( - (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.READ)) - ) - schedule.append( - (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.COMPUTE)) - ) - schedule.append( - (idx, DAGNodeOperation(bind_index, DAGNodeOperationType.WRITE)) - ) + for idx, _ in enumerate(self.actor_to_executable_tasks[actor_handle]): + schedule.append(DAGNodeOperation(idx, DAGNodeOperationType.READ)) + schedule.append(DAGNodeOperation(idx, DAGNodeOperationType.COMPUTE)) + schedule.append(DAGNodeOperation(idx, DAGNodeOperationType.WRITE)) self.actor_to_execution_schedule[actor_handle] = schedule def _detect_deadlock(self) -> bool: From e0332fbbfd07043626d70d1c07dceac77ef2ed2f Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 01:13:06 +0000 Subject: [PATCH 006/111] it somehow works --- python/ray/dag/compiled_dag_node.py | 197 +++++++++++++++++++++++++++- 1 file changed, 196 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 56ab28aeec298..f578341cf7f7e 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -389,6 +389,7 @@ def __init__( self.input_channels: List[ChannelInterface] = [] self.task_inputs: List[_ExecutableTaskInput] = [] self.resolved_kwargs: Dict[str, Any] = resolved_kwargs + self.idx = task.idx # Reverse map for input_channels: maps an input channel to # its index in input_channels. @@ -560,6 +561,9 @@ def __init__( self.actor_to_execution_schedule: Dict[ "ray.actor.ActorHandle", List[DAGNodeOperation] ] = defaultdict(list) + self.actor_to_execution_schedule_2: Dict[ + "ray.actor.ActorHandle", List[DAGNodeOperation] + ] = defaultdict(list) # Mapping from the actor handle to the node ID that the actor is on. self.actor_to_node_id: Dict["ray.actor.ActorHandle", str] = {} @@ -1067,7 +1071,8 @@ def _get_or_compile( self.actor_to_executable_tasks[actor_handle] = executable_tasks # Build schedule for each actor - self._build_execution_schedule() + # self._build_execution_schedule() + self._build_execution_schedule_2() # Execute schedule for each actor for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): @@ -1124,6 +1129,196 @@ def _build_execution_schedule(self): schedule.append(DAGNodeOperation(idx, DAGNodeOperationType.WRITE)) self.actor_to_execution_schedule[actor_handle] = schedule + def _build_execution_schedule_2(self): + from functools import total_ordering + + @total_ordering + class GraphNode: + def __init__(self, operation: DAGNodeOperation, idx, idx_to_task): + self.operation = operation + self.idx = idx + dag_node = idx_to_task[idx].dag_node + self.actor_handle = None + if isinstance(dag_node, ClassMethodNode): + self.actor_handle = dag_node._get_actor_handle() + self.requires_nccl = idx_to_task[idx].dag_node.type_hint.requires_nccl() + self.in_edges = set() + self.out_edges = set() + + @property + def in_degree(self) -> int: + return len(self.in_edges) + + def __lt__(self, other): + assert self.actor_handle == other.actor_handle + return self.operation.idx < other.operation.idx + + def __eq__(self, other): + assert self.actor_handle == other.actor_handle + if self.operation.idx == other.operation.idx: + return self.operation.type == other.operation.type + return False + + def __hash__(self): + return hash((self.operation, self.idx, self.operation.type)) + + def __repr__(self) -> str: + return ( + f"GraphNode(operation_type: {self.operation.type}, " + f"idx: {self.idx}, " + f"out_edges: {self.out_edges})" + ) + + def _add_edge(in_node: GraphNode, out_node: GraphNode): + in_node.out_edges.add((out_node.idx, out_node.operation.type)) + out_node.in_edges.add((in_node.idx, in_node.operation.type)) + + graph: Dict[int, Dict[DAGNodeOperationType, GraphNode]] = {} + + from ray.dag import ( + ClassMethodNode, + MultiOutputNode, + ) + + for _, executable_tasks in self.actor_to_executable_tasks.items(): + prev_compute_node = None + for local_idx, exec_task in enumerate(executable_tasks): + read_node = GraphNode( + DAGNodeOperation(local_idx, DAGNodeOperationType.READ), + exec_task.idx, + self.idx_to_task, + ) + compute_node = GraphNode( + DAGNodeOperation(local_idx, DAGNodeOperationType.COMPUTE), + exec_task.idx, + self.idx_to_task, + ) + write_node = GraphNode( + DAGNodeOperation(local_idx, DAGNodeOperationType.WRITE), + exec_task.idx, + self.idx_to_task, + ) + _add_edge(read_node, compute_node) + _add_edge(compute_node, write_node) + if prev_compute_node is not None: + _add_edge(prev_compute_node, compute_node) + prev_compute_node = compute_node + graph[exec_task.idx] = { + DAGNodeOperationType.READ: read_node, + DAGNodeOperationType.COMPUTE: compute_node, + DAGNodeOperationType.WRITE: write_node, + } + + for idx, task in self.idx_to_task.items(): + if not isinstance(task.dag_node, ClassMethodNode): + continue + for downstream_idx in task.downstream_node_idxs: + downstream_dag_node = self.idx_to_task[downstream_idx].dag_node + if isinstance(downstream_dag_node, MultiOutputNode): + continue + _add_edge( + graph[idx][DAGNodeOperationType.WRITE], + graph[downstream_idx][DAGNodeOperationType.READ], + ) + + # print("graph") + # for idx, node_dict in graph.items(): + # print("idx", idx) + # for _, node in node_dict.items(): + # print(node) + + actor_to_candidates = {} + for idx, node_dict in graph.items(): + for _, node in node_dict.items(): + if node.in_degree == 0: + if node.actor_handle not in actor_to_candidates: + actor_to_candidates[node.actor_handle] = [node] + else: + actor_to_candidates[node.actor_handle].append(node) + for actor_handle, candidates in actor_to_candidates.items(): + candidates.sort() + + # print("actor_to_candidates", actor_to_candidates) + + visited_nodes = set() + + def _select_next_nodes(): + next_nodes = [] + first_nccl_node = None + for _, candidates in actor_to_candidates.items(): + if ( + not candidates[0].requires_nccl + or candidates[0].operation.type != DAGNodeOperationType.WRITE + ): + next_nodes.append(candidates.pop(0)) + return next_nodes + if first_nccl_node is None: + first_nccl_node = candidates[0] + is_next_node = True + for downstream_node_metadata in candidates[0].out_edges: + downstream_node = graph[downstream_node_metadata[0]][ + downstream_node_metadata[1] + ] + downstream_node_actor = downstream_node.actor_handle + if ( + downstream_node_actor not in actor_to_candidates + or downstream_node + != actor_to_candidates[downstream_node_actor][0] + ): + is_next_node = False + break + if is_next_node: + next_nodes.append(candidates.pop(0)) + for downstream_node_metadata in next_nodes[0].out_edges: + downstream_node = graph[downstream_node_metadata[0]][ + downstream_node_metadata[1] + ] + next_nodes.append(downstream_node) + + next_nodes.append(first_nccl_node) + actor_to_candidates[first_nccl_node.actor_handle].remove(first_nccl_node) + + for downstream_node_metadata in first_nccl_node.out_edges: + downstream_node = graph[downstream_node_metadata[0]][ + downstream_node_metadata[1] + ] + next_nodes.append(downstream_node) + return next_nodes + + while actor_to_candidates: + nodes = _select_next_nodes() + for node in nodes: + if node in visited_nodes: + continue + self.actor_to_execution_schedule[node.actor_handle].append( + node.operation + ) + visited_nodes.add(node) + for out_node_idx, out_node_type in node.out_edges: + out_node = graph[out_node_idx][out_node_type] + out_node.in_edges.remove((node.idx, node.operation.type)) + if out_node.in_degree == 0: + if out_node.actor_handle not in actor_to_candidates: + actor_to_candidates[out_node.actor_handle] = [out_node] + else: + actor_to_candidates[out_node.actor_handle].append(out_node) + for _, candidates in actor_to_candidates.items(): + candidates.sort() + + delete_keys = [] + for actor_handle, candidates in actor_to_candidates.items(): + if len(candidates) == 0: + delete_keys.append(actor_handle) + for key in delete_keys: + del actor_to_candidates[key] + + # print("actor_to_execution_schedule") + # for actor_handle, schedule in self.actor_to_execution_schedule.items(): + # print("actor", actor_handle) + # print(schedule) + # print(self.actor_to_execution_schedule) + # TODO: Check whether topological sort exists or not. + def _detect_deadlock(self) -> bool: """ Create a graph with the following 3 rules, and then use From 4329e216a22b5ce818672d8ce58bde64b9f6ebdf Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 19:49:16 +0000 Subject: [PATCH 007/111] add a test --- python/ray/dag/BUILD | 1 + python/ray/dag/compiled_dag_node.py | 30 ++----- .../experimental/test_execution_schedule.py | 88 +++++++++++++++++++ 3 files changed, 94 insertions(+), 25 deletions(-) create mode 100644 python/ray/dag/tests/experimental/test_execution_schedule.py diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 013cc584cd1dd..7c7abfef21b56 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -103,6 +103,7 @@ py_test_module_list( size = "medium", files = [ "tests/experimental/test_detect_deadlock_dag.py", + "tests/experimental/test_execution_schedule.py", "tests/experimental/test_multi_node_dag.py", "tests/experimental/test_torch_tensor_dag.py", ], diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index f578341cf7f7e..03f6aa6296253 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1071,8 +1071,7 @@ def _get_or_compile( self.actor_to_executable_tasks[actor_handle] = executable_tasks # Build schedule for each actor - # self._build_execution_schedule() - self._build_execution_schedule_2() + self._build_execution_schedule() # Execute schedule for each actor for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): @@ -1121,15 +1120,6 @@ def _get_or_compile( self._dag_output_fetcher.start() def _build_execution_schedule(self): - for actor_handle in self.actor_to_executable_tasks: - schedule = [] - for idx, _ in enumerate(self.actor_to_executable_tasks[actor_handle]): - schedule.append(DAGNodeOperation(idx, DAGNodeOperationType.READ)) - schedule.append(DAGNodeOperation(idx, DAGNodeOperationType.COMPUTE)) - schedule.append(DAGNodeOperation(idx, DAGNodeOperationType.WRITE)) - self.actor_to_execution_schedule[actor_handle] = schedule - - def _build_execution_schedule_2(self): from functools import total_ordering @total_ordering @@ -1164,8 +1154,9 @@ def __hash__(self): def __repr__(self) -> str: return ( - f"GraphNode(operation_type: {self.operation.type}, " + f"GraphNode(operation: {self.operation}, " f"idx: {self.idx}, " + f"actor_handle: {self.actor_handle}, " f"out_edges: {self.out_edges})" ) @@ -1221,12 +1212,6 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): graph[downstream_idx][DAGNodeOperationType.READ], ) - # print("graph") - # for idx, node_dict in graph.items(): - # print("idx", idx) - # for _, node in node_dict.items(): - # print(node) - actor_to_candidates = {} for idx, node_dict in graph.items(): for _, node in node_dict.items(): @@ -1238,11 +1223,10 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): for actor_handle, candidates in actor_to_candidates.items(): candidates.sort() - # print("actor_to_candidates", actor_to_candidates) - visited_nodes = set() def _select_next_nodes(): + # TODO (kevin85421): Remove all nodes in next_nodes from actor_to_candidates next_nodes = [] first_nccl_node = None for _, candidates in actor_to_candidates.items(): @@ -1274,6 +1258,7 @@ def _select_next_nodes(): downstream_node_metadata[1] ] next_nodes.append(downstream_node) + return next_nodes next_nodes.append(first_nccl_node) actor_to_candidates[first_nccl_node.actor_handle].remove(first_nccl_node) @@ -1312,11 +1297,6 @@ def _select_next_nodes(): for key in delete_keys: del actor_to_candidates[key] - # print("actor_to_execution_schedule") - # for actor_handle, schedule in self.actor_to_execution_schedule.items(): - # print("actor", actor_handle) - # print(schedule) - # print(self.actor_to_execution_schedule) # TODO: Check whether topological sort exists or not. def _detect_deadlock(self) -> bool: diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py new file mode 100644 index 0000000000000..3a375b57e942c --- /dev/null +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -0,0 +1,88 @@ +# coding: utf-8 +import os +import sys + +import pytest + +import ray +import ray.cluster_utils +from ray.experimental.channel.torch_tensor_type import TorchTensorType +from ray.experimental.channel.conftest import start_nccl_mock +from ray.tests.conftest import * # noqa +from ray.dag import InputNode, MultiOutputNode + +if sys.platform != "linux" and sys.platform != "darwin": + pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) + + +@ray.remote(num_cpus=0, num_gpus=1) +class MockedWorker: + def __init__(self): + pass + + def start_mock(self): + """ + Patch methods that require CUDA. + """ + start_nccl_mock() + + def no_op(self, value): + return value + + def no_op_two(self, value1, value2): + return value1, value2 + + def fwd(self, value): + return value + + def bwd(self, value): + return value + + +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) +def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): + """ + If tensor_transport is TorchTensorType.AUTO, the shared memory channel will be + used, and the graph is valid. If tensor_transport is TorchTensorType.NCCL, the + NCCL channel will be used, and the graph is invalid. + + [Case: TorchTensorType.NCCL] + The first a.no_op writes to the second a.no_op via the NCCL channel. However, + the NCCL channel only supports synchronous communication and an actor can + only execute one task at a time, so the graph is deadlocked. + """ + monkeypatch.setattr(ray.dag.constants, 'RAY_ADAG_ENABLE_DETECT_DEADLOCK', False) + + w1 = MockedWorker.remote() + w2 = MockedWorker.remote() + + ray.get([w1.start_mock.remote(), w2.start_mock.remote()]) + + with InputNode() as inp: + batch_1 = w1.fwd.bind(inp) + batch_1.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_2 = w1.fwd.bind(inp) + batch_2.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_1 = w2.fwd.bind(batch_1) + batch_1 = w2.bwd.bind(batch_1) + batch_1.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_2 = w2.fwd.bind(batch_2) + batch_1 = w1.bwd.bind(batch_1) + batch_2 = w2.bwd.bind(batch_2) + batch_2.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_2 = w1.bwd.bind(batch_2) + dag = MultiOutputNode( + [ + batch_1, + batch_2, + ] + ) + compiled_graph = dag.experimental_compile() + compiled_graph.teardown() + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__ + "::test_valid_graph_3_actors"])) From 1d6756a7e2644964792cf94852efa4c628ea4bf6 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 20:45:53 +0000 Subject: [PATCH 008/111] polish --- python/ray/dag/compiled_dag_node.py | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 03f6aa6296253..02a57f83d31a1 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1127,11 +1127,12 @@ class GraphNode: def __init__(self, operation: DAGNodeOperation, idx, idx_to_task): self.operation = operation self.idx = idx + dag_node = idx_to_task[idx].dag_node self.actor_handle = None if isinstance(dag_node, ClassMethodNode): self.actor_handle = dag_node._get_actor_handle() - self.requires_nccl = idx_to_task[idx].dag_node.type_hint.requires_nccl() + self.requires_nccl = dag_node.type_hint.requires_nccl() self.in_edges = set() self.out_edges = set() @@ -1212,14 +1213,12 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): graph[downstream_idx][DAGNodeOperationType.READ], ) - actor_to_candidates = {} + actor_to_candidates = defaultdict(list) for idx, node_dict in graph.items(): for _, node in node_dict.items(): if node.in_degree == 0: - if node.actor_handle not in actor_to_candidates: - actor_to_candidates[node.actor_handle] = [node] - else: - actor_to_candidates[node.actor_handle].append(node) + actor_to_candidates[node.actor_handle].append(node) + for actor_handle, candidates in actor_to_candidates.items(): candidates.sort() @@ -1283,10 +1282,7 @@ def _select_next_nodes(): out_node = graph[out_node_idx][out_node_type] out_node.in_edges.remove((node.idx, node.operation.type)) if out_node.in_degree == 0: - if out_node.actor_handle not in actor_to_candidates: - actor_to_candidates[out_node.actor_handle] = [out_node] - else: - actor_to_candidates[out_node.actor_handle].append(out_node) + actor_to_candidates[out_node.actor_handle].append(out_node) for _, candidates in actor_to_candidates.items(): candidates.sort() From 2174bb69795ff9091bc8754ca8eeb3baaf87b550 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 21:09:01 +0000 Subject: [PATCH 009/111] polish --- python/ray/dag/compiled_dag_node.py | 24 +++++++++---------- .../experimental/test_execution_schedule.py | 2 +- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 02a57f83d31a1..1ca6ac7e1aded 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1213,14 +1213,13 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): graph[downstream_idx][DAGNodeOperationType.READ], ) + import heapq + actor_to_candidates = defaultdict(list) for idx, node_dict in graph.items(): for _, node in node_dict.items(): if node.in_degree == 0: - actor_to_candidates[node.actor_handle].append(node) - - for actor_handle, candidates in actor_to_candidates.items(): - candidates.sort() + heapq.heappush(actor_to_candidates[node.actor_handle], node) visited_nodes = set() @@ -1233,7 +1232,7 @@ def _select_next_nodes(): not candidates[0].requires_nccl or candidates[0].operation.type != DAGNodeOperationType.WRITE ): - next_nodes.append(candidates.pop(0)) + next_nodes.append(heapq.heappop(candidates)) return next_nodes if first_nccl_node is None: first_nccl_node = candidates[0] @@ -1251,7 +1250,7 @@ def _select_next_nodes(): is_next_node = False break if is_next_node: - next_nodes.append(candidates.pop(0)) + next_nodes.append(heapq.heappop(candidates)) for downstream_node_metadata in next_nodes[0].out_edges: downstream_node = graph[downstream_node_metadata[0]][ downstream_node_metadata[1] @@ -1259,9 +1258,10 @@ def _select_next_nodes(): next_nodes.append(downstream_node) return next_nodes - next_nodes.append(first_nccl_node) - actor_to_candidates[first_nccl_node.actor_handle].remove(first_nccl_node) - + assert first_nccl_node is not None + next_nodes.append( + heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle]) + ) for downstream_node_metadata in first_nccl_node.out_edges: downstream_node = graph[downstream_node_metadata[0]][ downstream_node_metadata[1] @@ -1282,9 +1282,9 @@ def _select_next_nodes(): out_node = graph[out_node_idx][out_node_type] out_node.in_edges.remove((node.idx, node.operation.type)) if out_node.in_degree == 0: - actor_to_candidates[out_node.actor_handle].append(out_node) - for _, candidates in actor_to_candidates.items(): - candidates.sort() + heapq.heappush( + actor_to_candidates[out_node.actor_handle], out_node + ) delete_keys = [] for actor_handle, candidates in actor_to_candidates.items(): diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 3a375b57e942c..90ac46ae1e778 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -51,7 +51,7 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): the NCCL channel only supports synchronous communication and an actor can only execute one task at a time, so the graph is deadlocked. """ - monkeypatch.setattr(ray.dag.constants, 'RAY_ADAG_ENABLE_DETECT_DEADLOCK', False) + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) w1 = MockedWorker.remote() w2 = MockedWorker.remote() From ea7685b46726e76bfe1a06c173ee2fe2a4d4aeb5 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 21:38:57 +0000 Subject: [PATCH 010/111] polish --- python/ray/dag/compiled_dag_node.py | 22 ++++++++----------- .../experimental/test_execution_schedule.py | 2 +- 2 files changed, 10 insertions(+), 14 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 1ca6ac7e1aded..4ab9270acf34f 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -127,10 +127,8 @@ def do_exec_tasks( if done: break for operation in schedule: - operation_type = operation.type - idx = operation.idx - task = tasks[idx] - done = _exec_task(self, task, idx, operation_type) + task = tasks[operation.idx] + done = _exec_operation(self, task, operation) if done: break except Exception: @@ -175,20 +173,18 @@ def _wrap_exception(exc): return wrapped -def _exec_task( - self, task: "ExecutableTask", idx: int, op_type: DAGNodeOperationType -) -> bool: +def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) -> bool: """ - Execute the task. + Execute the `operation` which belongs to `task`. Args: task: The task to execute. - idx: The index of the task in the list of tasks of the actor. + operation: The operation to execute. Returns: - True if we are done executing all tasks of this actor, False otherwise. + True if we are done executing all operations of this actor, False otherwise. """ - # TODO: for cases where output is passed as input to a task on - # the same actor, introduce a "IntraProcessChannel" to avoid the overhead - # of serialization/deserialization and synchronization. + idx = operation.idx + op_type = operation.type + input_reader = self._input_readers[idx] output_writer = self._output_writers[idx] ctx = ChannelContext.get_current().serialization_context diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 90ac46ae1e778..6df9da150474e 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -85,4 +85,4 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) else: - sys.exit(pytest.main(["-sv", __file__ + "::test_valid_graph_3_actors"])) + sys.exit(pytest.main(["-sv", __file__])) From a98b1eb28ba102a9a136bf71d32a0d78716c2678 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 21:49:46 +0000 Subject: [PATCH 011/111] polish --- .../channel/serialization_context.py | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/python/ray/experimental/channel/serialization_context.py b/python/ray/experimental/channel/serialization_context.py index c51d104fd4660..b064760f7ed53 100644 --- a/python/ray/experimental/channel/serialization_context.py +++ b/python/ray/experimental/channel/serialization_context.py @@ -18,26 +18,28 @@ def __init__(self): # The number of readers for each channel. When the number of readers # reaches 0, remove the data from the buffer. self.channel_id_to_num_readers: Dict[str, int] = {} + # The key is the order of the task on the actor, which follows the ascending + # order of bind_index, and the value is a dictionary of intermediate results. + # The dictionary maps the operation type, such as "READ", "COMPUTE", and "WRITE" + # , to the intermediate result. self.intermediate_results_buffer: Dict[int, Dict[str, Any]] = {} def set_use_external_transport(self, use_external_transport: bool) -> None: self.use_external_transport = use_external_transport - def set_intermediate_result( - self, bind_index: int, op_type: str, value: Any - ) -> None: - if bind_index not in self.intermediate_results_buffer: - self.intermediate_results_buffer[bind_index] = {} - self.intermediate_results_buffer[bind_index][op_type] = value + def set_intermediate_result(self, idx: int, op_type: str, value: Any) -> None: + if idx not in self.intermediate_results_buffer: + self.intermediate_results_buffer[idx] = {} + self.intermediate_results_buffer[idx][op_type] = value - def get_intermediate_result(self, bind_index: int, op_type: str) -> Any: + def get_intermediate_result(self, idx: int, op_type: str) -> Any: assert ( - bind_index in self.intermediate_results_buffer - ), f"Bind index {bind_index} does not exist in the buffer." + idx in self.intermediate_results_buffer + ), f"Index {idx} does not exist in the buffer." assert ( - op_type in self.intermediate_results_buffer[bind_index] + op_type in self.intermediate_results_buffer[idx] ), f"Operation type {op_type} does not exist in the buffer." - return self.intermediate_results_buffer[bind_index].pop(op_type) + return self.intermediate_results_buffer[idx].pop(op_type) def set_data(self, channel_id: str, value: Any, num_readers: int) -> None: assert num_readers > 0, "num_readers must be greater than 0." From ecefb26739a468291de9b8960dd160aeae8899b5 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 21:57:51 +0000 Subject: [PATCH 012/111] polish --- python/ray/dag/compiled_dag_node.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 4ab9270acf34f..036bf30bae5b5 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -215,7 +215,6 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - try: output_val = method(*resolved_inputs, **task.resolved_kwargs) - # TODO: Cache in buffer. except Exception as exc: output_val = _wrap_exception(exc) ctx.set_intermediate_result(idx, op_type.value, output_val) @@ -557,9 +556,6 @@ def __init__( self.actor_to_execution_schedule: Dict[ "ray.actor.ActorHandle", List[DAGNodeOperation] ] = defaultdict(list) - self.actor_to_execution_schedule_2: Dict[ - "ray.actor.ActorHandle", List[DAGNodeOperation] - ] = defaultdict(list) # Mapping from the actor handle to the node ID that the actor is on. self.actor_to_node_id: Dict["ray.actor.ActorHandle", str] = {} From 088384e7d9503ea438df4095a33bfe36fb2656f0 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 22:06:07 +0000 Subject: [PATCH 013/111] polish --- python/ray/dag/compiled_dag_node.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 036bf30bae5b5..28bb6b33ac9ce 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -553,6 +553,8 @@ def __init__( self.actor_to_executable_tasks: Dict[ "ray.actor.ActorHandle", List["ExecutableTask"] ] = {} + # Mapping from the actor handle to the execution schedule which is a list + # of operations to be executed. self.actor_to_execution_schedule: Dict[ "ray.actor.ActorHandle", List[DAGNodeOperation] ] = defaultdict(list) @@ -1062,10 +1064,8 @@ def _get_or_compile( executable_tasks.sort(key=lambda task: task.bind_index) self.actor_to_executable_tasks[actor_handle] = executable_tasks - # Build schedule for each actor + # Build an execution schedule for each actor self._build_execution_schedule() - - # Execute schedule for each actor for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): self.worker_task_refs[actor_handle] = actor_handle.__ray_call__.options( concurrency_group="_ray_system" From b2ac0f92d298cada45fe44afed88bd244c2da972 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 22:15:25 +0000 Subject: [PATCH 014/111] polish --- .../experimental/test_execution_schedule.py | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 6df9da150474e..9170bd8529bc1 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -26,12 +26,6 @@ def start_mock(self): """ start_nccl_mock() - def no_op(self, value): - return value - - def no_op_two(self, value1, value2): - return value1, value2 - def fwd(self, value): return value @@ -42,14 +36,14 @@ def bwd(self, value): @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): """ - If tensor_transport is TorchTensorType.AUTO, the shared memory channel will be - used, and the graph is valid. If tensor_transport is TorchTensorType.NCCL, the - NCCL channel will be used, and the graph is invalid. - - [Case: TorchTensorType.NCCL] - The first a.no_op writes to the second a.no_op via the NCCL channel. However, - the NCCL channel only supports synchronous communication and an actor can - only execute one task at a time, so the graph is deadlocked. + This test simulates a simple 1F1B pipeline parallelism for training with + 2 workers and 2 batches. + + w1: fwd_b1 fwd_b2 bwd_b1 bwd_b2 + w2: fwd_b1 bwd_b1 fwd_b2 bwd_b2 + + The communication between workers is done using NCCL. The communication + within the worker actor is done using IntraProcessChannel. """ monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) From bc909661e58b0d92e7a12d09f9ba4725b9a9f7f1 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 22:31:31 +0000 Subject: [PATCH 015/111] polish --- .../experimental/test_execution_schedule.py | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 9170bd8529bc1..5be7cbe433c71 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -10,6 +10,7 @@ from ray.experimental.channel.conftest import start_nccl_mock from ray.tests.conftest import * # noqa from ray.dag import InputNode, MultiOutputNode +from ray.dag.compiled_dag_node import DAGNodeOperationType if sys.platform != "linux" and sys.platform != "darwin": pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) @@ -72,6 +73,45 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): ] ) compiled_graph = dag.experimental_compile() + + w1_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (0, DAGNodeOperationType.WRITE), + (1, DAGNodeOperationType.READ), + (1, DAGNodeOperationType.COMPUTE), + (1, DAGNodeOperationType.WRITE), + (2, DAGNodeOperationType.READ), + (2, DAGNodeOperationType.COMPUTE), + (2, DAGNodeOperationType.WRITE), + (3, DAGNodeOperationType.READ), + (3, DAGNodeOperationType.COMPUTE), + (3, DAGNodeOperationType.WRITE), + ] + w2_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (0, DAGNodeOperationType.WRITE), + (1, DAGNodeOperationType.READ), + (1, DAGNodeOperationType.COMPUTE), + (2, DAGNodeOperationType.READ), + (1, DAGNodeOperationType.WRITE), + (2, DAGNodeOperationType.COMPUTE), + (2, DAGNodeOperationType.WRITE), + (3, DAGNodeOperationType.READ), + (3, DAGNodeOperationType.COMPUTE), + (3, DAGNodeOperationType.WRITE), + ] + w1_schedule = compiled_graph.actor_to_execution_schedule[w1] + w2_schedule = compiled_graph.actor_to_execution_schedule[w2] + + for schedule, expected_schedule in zip( + [w1_schedule, w2_schedule], [w1_expected_schedule, w2_expected_schedule] + ): + assert len(schedule) == len(expected_schedule) + for i, operation in enumerate(schedule): + assert operation.idx == expected_schedule[i][0] + assert operation.type == expected_schedule[i][1] compiled_graph.teardown() From ff701e2540eca1d5c083c3b6377fbac83530d1f4 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 22:43:44 +0000 Subject: [PATCH 016/111] polish --- python/ray/dag/compiled_dag_node.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 28bb6b33ac9ce..e6b91e8702068 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -43,6 +43,12 @@ class DAGNodeOperationType(Enum): + """ + There are three types of operations that a DAG node can perform: + 1. READ: Read from an input channel. + 2. COMPUTE: Execute the method corresponding to the node. + 3. WRITE: Write to an output channel. + """ READ = "READ" COMPUTE = "COMPUTE" WRITE = "WRITE" @@ -54,13 +60,15 @@ def __init__( idx: int, operation_type: DAGNodeOperationType, ): - # not bind_index + """ + Args: + idx: The index of the task that this operation belongs to + in the actor's ExecutableTask list. + operation_type: The type of operation to perform. + """ self.idx = idx self.type = operation_type - def __repr__(self) -> str: - return f"DAGNodeOperation({self.idx}, {self.type})" - # Holds the input arguments for an accelerated DAG node. @PublicAPI(stability="alpha") From 53f5add02c6c47ca7d2f93098d4f935521e3191b Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 22:46:21 +0000 Subject: [PATCH 017/111] polish --- python/ray/dag/compiled_dag_node.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index e6b91e8702068..aca290631c97c 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -49,6 +49,7 @@ class DAGNodeOperationType(Enum): 2. COMPUTE: Execute the method corresponding to the node. 3. WRITE: Write to an output channel. """ + READ = "READ" COMPUTE = "COMPUTE" WRITE = "WRITE" @@ -63,7 +64,9 @@ def __init__( """ Args: idx: The index of the task that this operation belongs to - in the actor's ExecutableTask list. + in the actor's ExecutableTask list. The index is not + the same as bind_index, but there are positive correlations + between the two. operation_type: The type of operation to perform. """ self.idx = idx From 341915112c5e322ea31d86f40f8698eda40781b9 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 23:05:53 +0000 Subject: [PATCH 018/111] polish --- python/ray/dag/compiled_dag_node.py | 26 ++++++++++++-------------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index aca290631c97c..5a8634aa7a8f5 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1127,14 +1127,11 @@ def _build_execution_schedule(self): @total_ordering class GraphNode: - def __init__(self, operation: DAGNodeOperation, idx, idx_to_task): + def __init__(self, operation: DAGNodeOperation, idx, dag_node): self.operation = operation self.idx = idx - - dag_node = idx_to_task[idx].dag_node - self.actor_handle = None - if isinstance(dag_node, ClassMethodNode): - self.actor_handle = dag_node._get_actor_handle() + assert isinstance(dag_node, ClassMethodNode) + self.actor_handle = dag_node._get_actor_handle() self.requires_nccl = dag_node.type_hint.requires_nccl() self.in_edges = set() self.out_edges = set() @@ -1154,7 +1151,7 @@ def __eq__(self, other): return False def __hash__(self): - return hash((self.operation, self.idx, self.operation.type)) + return hash((self.operation, self.idx)) def __repr__(self) -> str: return ( @@ -1178,27 +1175,28 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): for _, executable_tasks in self.actor_to_executable_tasks.items(): prev_compute_node = None for local_idx, exec_task in enumerate(executable_tasks): + idx = exec_task.idx read_node = GraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.READ), - exec_task.idx, - self.idx_to_task, + idx, + self.idx_to_task[idx].dag_node, ) compute_node = GraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.COMPUTE), - exec_task.idx, - self.idx_to_task, + idx, + self.idx_to_task[idx].dag_node, ) write_node = GraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.WRITE), - exec_task.idx, - self.idx_to_task, + idx, + self.idx_to_task[idx].dag_node, ) _add_edge(read_node, compute_node) _add_edge(compute_node, write_node) if prev_compute_node is not None: _add_edge(prev_compute_node, compute_node) prev_compute_node = compute_node - graph[exec_task.idx] = { + graph[idx] = { DAGNodeOperationType.READ: read_node, DAGNodeOperationType.COMPUTE: compute_node, DAGNodeOperationType.WRITE: write_node, From 3a8acb0840d4be63bb0a047863045398007acb40 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 23:48:00 +0000 Subject: [PATCH 019/111] polish --- python/ray/dag/compiled_dag_node.py | 54 ++++++++++++++++++++++++----- 1 file changed, 45 insertions(+), 9 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 5a8634aa7a8f5..11dbe9154f64e 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1123,6 +1123,44 @@ def _get_or_compile( self._dag_output_fetcher.start() def _build_execution_schedule(self): + """ + Generate an execution schedule for each actor. The schedule is a list of + DAGNodeOperation. + + Step 1: Generate a graph based on the following rules: + + #1 Divide a DAG node into three GraphNodes: READ, COMPUTE, and WRITE. Each + GraphNode has a DAGNodeOperation. + #2 Add edges between READ and COMPUTE, and between COMPUTE and WRITE, which + belong to the same task. + #3 Add an edge between COMPUTE with bind_index i and COMPUTE with bind_index + i+1 if they belong to the same actor. + #4 Add an edge between WRITE of the writer task and READ of the reader task. + + Step 2: Topological sort + + If there are multiple GraphNodes with zero in-degree, select one based on + the following rules: + + #1 If the nodes are not NCCL write nodes, select the one with the smallest + `bind_index`. If there are multiple candidate nodes with the smallest + `bind_index` of the actors that they belong to, any one of them is + acceptable. + + #2 If the node is an NCCL write node, select it only if all of its downstream + nodes are also the roots of their heaps. + + #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are + NCCL write nodes. In this case, select the one that is the root of the + heap and its downstream nodes, regardless of whether the downstream nodes + are roots of their heaps or not. + + Then, put the selected nodes into the corresponding actors' schedules. + + [Example]: + + See `test_execution_schedule` for more examples. + """ from functools import total_ordering @total_ordering @@ -1153,14 +1191,6 @@ def __eq__(self, other): def __hash__(self): return hash((self.operation, self.idx)) - def __repr__(self) -> str: - return ( - f"GraphNode(operation: {self.operation}, " - f"idx: {self.idx}, " - f"actor_handle: {self.actor_handle}, " - f"out_edges: {self.out_edges})" - ) - def _add_edge(in_node: GraphNode, out_node: GraphNode): in_node.out_edges.add((out_node.idx, out_node.operation.type)) out_node.in_edges.add((in_node.idx, in_node.operation.type)) @@ -1225,7 +1255,13 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): visited_nodes = set() def _select_next_nodes(): - # TODO (kevin85421): Remove all nodes in next_nodes from actor_to_candidates + """ + Select the next nodes for topological sort. This function may return + multiple nodes if they are NCCL nodes. In that case, this function only + removes the NCCL write node, which is also the root of a heap. Other nodes + will be removed in the following iterations. Additionally, visited_nodes + ensures that the same node will not be scheduled more than once. + """ next_nodes = [] first_nccl_node = None for _, candidates in actor_to_candidates.items(): From c0e6cb75fcee47114b25abbf5faee68159dceb6d Mon Sep 17 00:00:00 2001 From: kaihsun Date: Wed, 31 Jul 2024 23:52:08 +0000 Subject: [PATCH 020/111] polish --- python/ray/dag/compiled_dag_node.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 11dbe9154f64e..50a769b14fdb7 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1146,10 +1146,8 @@ def _build_execution_schedule(self): `bind_index`. If there are multiple candidate nodes with the smallest `bind_index` of the actors that they belong to, any one of them is acceptable. - #2 If the node is an NCCL write node, select it only if all of its downstream nodes are also the roots of their heaps. - #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are NCCL write nodes. In this case, select the one that is the root of the heap and its downstream nodes, regardless of whether the downstream nodes @@ -1161,6 +1159,10 @@ def _build_execution_schedule(self): See `test_execution_schedule` for more examples. """ + assert self.idx_to_task + assert self.actor_to_executable_tasks + assert not self.actor_to_execution_schedule + from functools import total_ordering @total_ordering From b8b8ea531dd7f915ef238f630fc7f971af61623c Mon Sep 17 00:00:00 2001 From: kaihsun Date: Thu, 1 Aug 2024 00:02:31 +0000 Subject: [PATCH 021/111] add new tests --- .../experimental/test_execution_schedule.py | 72 +++++++++++++++++-- 1 file changed, 68 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 5be7cbe433c71..3e2e874cba84a 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -33,6 +33,12 @@ def fwd(self, value): def bwd(self, value): return value + def no_op(self, value): + return value + + def no_op_two(self, value1, value2): + return value1, value2 + @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): @@ -72,7 +78,7 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): batch_2, ] ) - compiled_graph = dag.experimental_compile() + compiled_dag = dag.experimental_compile() w1_expected_schedule = [ (0, DAGNodeOperationType.READ), @@ -102,8 +108,8 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): (3, DAGNodeOperationType.COMPUTE), (3, DAGNodeOperationType.WRITE), ] - w1_schedule = compiled_graph.actor_to_execution_schedule[w1] - w2_schedule = compiled_graph.actor_to_execution_schedule[w2] + w1_schedule = compiled_dag.actor_to_execution_schedule[w1] + w2_schedule = compiled_dag.actor_to_execution_schedule[w2] for schedule, expected_schedule in zip( [w1_schedule, w2_schedule], [w1_expected_schedule, w2_expected_schedule] @@ -112,7 +118,65 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): for i, operation in enumerate(schedule): assert operation.idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] - compiled_graph.teardown() + compiled_dag.teardown() + + +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) +def test_3_actors_with_nccl(ray_start_regular): + """ + Driver -> a.no_op -> b.no_op -> a.no_op_two -> Driver + | | + -> c.no_op - + """ + a = MockedWorker.remote() + b = MockedWorker.remote() + c = MockedWorker.remote() + + ray.get([a.start_mock.remote(), b.start_mock.remote(), c.start_mock.remote()]) + + with InputNode() as inp: + dag = a.no_op.bind(inp) + dag.with_type_hint(TorchTensorType(transport="nccl")) + branch1 = b.no_op.bind(dag) + branch1.with_type_hint(TorchTensorType(transport="nccl")) + branch2 = c.no_op.bind(dag) + branch2.with_type_hint(TorchTensorType(transport="nccl")) + dag = a.no_op_two.bind(branch1, branch2) + + compiled_dag = dag.experimental_compile() + + a_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (0, DAGNodeOperationType.WRITE), + (1, DAGNodeOperationType.READ), + (1, DAGNodeOperationType.COMPUTE), + (1, DAGNodeOperationType.WRITE), + ] + b_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (0, DAGNodeOperationType.WRITE), + ] + c_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (0, DAGNodeOperationType.WRITE), + ] + a_schedule = compiled_dag.actor_to_execution_schedule[a] + b_schedule = compiled_dag.actor_to_execution_schedule[b] + c_schedule = compiled_dag.actor_to_execution_schedule[c] + + for schedule, expected_schedule in zip( + [a_schedule, b_schedule, c_schedule], + [a_expected_schedule, b_expected_schedule, c_expected_schedule], + ): + assert len(schedule) == len(expected_schedule) + for i, operation in enumerate(schedule): + assert operation.idx == expected_schedule[i][0] + assert operation.type == expected_schedule[i][1] + + compiled_dag.teardown() if __name__ == "__main__": From ce7f0e5da80f5854f1d94984c287c7b288d03735 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Thu, 1 Aug 2024 02:37:58 +0000 Subject: [PATCH 022/111] add new tests --- .../experimental/test_execution_schedule.py | 70 ++++++++++++++++++- 1 file changed, 69 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 3e2e874cba84a..f57473e5059d5 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -122,7 +122,7 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) -def test_3_actors_with_nccl(ray_start_regular): +def test_three_actors_with_nccl_1(ray_start_regular): """ Driver -> a.no_op -> b.no_op -> a.no_op_two -> Driver | | @@ -179,6 +179,74 @@ def test_3_actors_with_nccl(ray_start_regular): compiled_dag.teardown() +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) +def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) + + a = MockedWorker.remote() + b = MockedWorker.remote() + c = MockedWorker.remote() + + ray.get([a.start_mock.remote(), b.start_mock.remote(), c.start_mock.remote()]) + + with InputNode() as inp: + branch1 = a.no_op.bind(inp) + branch1.with_type_hint(TorchTensorType(transport="nccl")) + branch2 = b.no_op.bind(inp) + branch2.with_type_hint(TorchTensorType(transport="nccl")) + branch3 = c.no_op.bind(inp) + branch3.with_type_hint(TorchTensorType(transport="nccl")) + dag = MultiOutputNode( + [ + a.no_op.bind(branch3), + b.no_op.bind(branch1), + c.no_op.bind(branch2), + ] + ) + + compiled_dag = dag.experimental_compile() + + a_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (1, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.WRITE), + (1, DAGNodeOperationType.COMPUTE), + (1, DAGNodeOperationType.WRITE), + ] + b_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (1, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.WRITE), + (1, DAGNodeOperationType.COMPUTE), + (1, DAGNodeOperationType.WRITE), + ] + c_expected_schedule = [ + (0, DAGNodeOperationType.READ), + (0, DAGNodeOperationType.COMPUTE), + (0, DAGNodeOperationType.WRITE), + (1, DAGNodeOperationType.READ), + (1, DAGNodeOperationType.COMPUTE), + (1, DAGNodeOperationType.WRITE), + ] + + a_schedule = compiled_dag.actor_to_execution_schedule[a] + b_schedule = compiled_dag.actor_to_execution_schedule[b] + c_schedule = compiled_dag.actor_to_execution_schedule[c] + + for schedule, expected_schedule in zip( + [a_schedule, b_schedule, c_schedule], + [a_expected_schedule, b_expected_schedule, c_expected_schedule], + ): + assert len(schedule) == len(expected_schedule) + for i, operation in enumerate(schedule): + assert operation.idx == expected_schedule[i][0] + assert operation.type == expected_schedule[i][1] + + compiled_dag.teardown() + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From 900e7900bdf9abd89576900aa008fcaba03e7673 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Thu, 1 Aug 2024 02:47:07 +0000 Subject: [PATCH 023/111] add new tests --- .../experimental/test_execution_schedule.py | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index f57473e5059d5..6a8efeb4baf81 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -33,6 +33,9 @@ def fwd(self, value): def bwd(self, value): return value + def read_input(self, input): + return input + def no_op(self, value): return value @@ -60,9 +63,10 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): ray.get([w1.start_mock.remote(), w2.start_mock.remote()]) with InputNode() as inp: - batch_1 = w1.fwd.bind(inp) + w1_input = w1.read_input.bind(inp) + batch_1 = w1.fwd.bind(w1_input) batch_1.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) - batch_2 = w1.fwd.bind(inp) + batch_2 = w1.fwd.bind(w1_input) batch_2.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) batch_1 = w2.fwd.bind(batch_1) batch_1 = w2.bwd.bind(batch_1) @@ -93,6 +97,9 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): (3, DAGNodeOperationType.READ), (3, DAGNodeOperationType.COMPUTE), (3, DAGNodeOperationType.WRITE), + (4, DAGNodeOperationType.READ), + (4, DAGNodeOperationType.COMPUTE), + (4, DAGNodeOperationType.WRITE), ] w2_expected_schedule = [ (0, DAGNodeOperationType.READ), @@ -118,6 +125,10 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): for i, operation in enumerate(schedule): assert operation.idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] + + ref = compiled_dag.execute(123) + assert ray.get(ref) == [123, 123] + compiled_dag.teardown() @@ -176,6 +187,9 @@ def test_three_actors_with_nccl_1(ray_start_regular): assert operation.idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] + ref = compiled_dag.execute(123) + assert ray.get(ref) == (123, 123) + compiled_dag.teardown() @@ -244,6 +258,9 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): assert operation.idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] + ref = compiled_dag.execute(123) + assert ray.get(ref) == [123, 123, 123] + compiled_dag.teardown() From e36277e1b941560a78db440ffeceee236e3a255e Mon Sep 17 00:00:00 2001 From: kaihsun Date: Thu, 1 Aug 2024 03:43:52 +0000 Subject: [PATCH 024/111] add developer api --- python/ray/dag/compiled_dag_node.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 50a769b14fdb7..d0c33d1427e2f 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -42,6 +42,7 @@ from enum import Enum +@DeveloperAPI class DAGNodeOperationType(Enum): """ There are three types of operations that a DAG node can perform: @@ -55,6 +56,7 @@ class DAGNodeOperationType(Enum): WRITE = "WRITE" +@DeveloperAPI class DAGNodeOperation: def __init__( self, From 83f6209f07d3f8505d98a4d1b7f57ab97a351200 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Thu, 1 Aug 2024 19:59:28 +0000 Subject: [PATCH 025/111] update comment Signed-off-by: kaihsun --- python/ray/dag/compiled_dag_node.py | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index d0c33d1427e2f..00332b95fc1e2 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1147,13 +1147,15 @@ def _build_execution_schedule(self): #1 If the nodes are not NCCL write nodes, select the one with the smallest `bind_index`. If there are multiple candidate nodes with the smallest `bind_index` of the actors that they belong to, any one of them is - acceptable. + acceptable. For the implementation details, we maintain a priority queue + for each actor, where the peek of the priority queue is the node with the + smallest `bind_index`. #2 If the node is an NCCL write node, select it only if all of its downstream - nodes are also the roots of their heaps. + nodes are also the peeks of their priority queues. #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are - NCCL write nodes. In this case, select the one that is the root of the - heap and its downstream nodes, regardless of whether the downstream nodes - are roots of their heaps or not. + NCCL write nodes. In this case, select the one that is the peek of the + priority queue and its downstream nodes, regardless of whether the + downstream nodes are peeks of their priority queues or not. Then, put the selected nodes into the corresponding actors' schedules. @@ -1262,9 +1264,10 @@ def _select_next_nodes(): """ Select the next nodes for topological sort. This function may return multiple nodes if they are NCCL nodes. In that case, this function only - removes the NCCL write node, which is also the root of a heap. Other nodes - will be removed in the following iterations. Additionally, visited_nodes - ensures that the same node will not be scheduled more than once. + removes the NCCL write node, which is also the peek of a priority queue. + Other nodes will be removed in the following iterations. Additionally, + visited_nodes ensures that the same node will not be scheduled more than + once. """ next_nodes = [] first_nccl_node = None From 8a8e75ad2c7dea53bc01ae2398a202d23b828bc3 Mon Sep 17 00:00:00 2001 From: kaihsun Date: Fri, 2 Aug 2024 23:18:58 +0000 Subject: [PATCH 026/111] Add GPU tests Signed-off-by: kaihsun --- .../experimental/test_execution_schedule.py | 71 ++++++++++++------- 1 file changed, 44 insertions(+), 27 deletions(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 6a8efeb4baf81..a500c4a5770a1 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -7,26 +7,22 @@ import ray import ray.cluster_utils from ray.experimental.channel.torch_tensor_type import TorchTensorType -from ray.experimental.channel.conftest import start_nccl_mock from ray.tests.conftest import * # noqa from ray.dag import InputNode, MultiOutputNode from ray.dag.compiled_dag_node import DAGNodeOperationType +import torch if sys.platform != "linux" and sys.platform != "darwin": pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) +USE_GPU = bool(os.environ.get("RAY_PYTEST_USE_GPU", 0)) + @ray.remote(num_cpus=0, num_gpus=1) -class MockedWorker: +class Worker: def __init__(self): pass - def start_mock(self): - """ - Patch methods that require CUDA. - """ - start_nccl_mock() - def fwd(self, value): return value @@ -55,12 +51,13 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): The communication between workers is done using NCCL. The communication within the worker actor is done using IntraProcessChannel. """ - monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") - w1 = MockedWorker.remote() - w2 = MockedWorker.remote() + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) - ray.get([w1.start_mock.remote(), w2.start_mock.remote()]) + w1 = Worker.remote() + w2 = Worker.remote() with InputNode() as inp: w1_input = w1.read_input.bind(inp) @@ -126,8 +123,14 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): assert operation.idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] - ref = compiled_dag.execute(123) - assert ray.get(ref) == [123, 123] + tensor_cpu = torch.zeros(10, 10) + ref = compiled_dag.execute(tensor_cpu) + tensors = ray.get(ref) + tensor_cuda = tensor_cpu.to("cuda:0") + + assert len(tensors) == 2 + for t in tensors: + assert torch.equal(t, tensor_cuda) compiled_dag.teardown() @@ -139,11 +142,12 @@ def test_three_actors_with_nccl_1(ray_start_regular): | | -> c.no_op - """ - a = MockedWorker.remote() - b = MockedWorker.remote() - c = MockedWorker.remote() + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") - ray.get([a.start_mock.remote(), b.start_mock.remote(), c.start_mock.remote()]) + a = Worker.remote() + b = Worker.remote() + c = Worker.remote() with InputNode() as inp: dag = a.no_op.bind(inp) @@ -187,21 +191,28 @@ def test_three_actors_with_nccl_1(ray_start_regular): assert operation.idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] - ref = compiled_dag.execute(123) - assert ray.get(ref) == (123, 123) + tensor_cpu = torch.zeros(10, 10) + ref = compiled_dag.execute(tensor_cpu) + tensors = ray.get(ref) + tensor_cuda = tensor_cpu.to("cuda:0") + + assert len(tensors) == 2 + for t in tensors: + assert torch.equal(t, tensor_cuda) compiled_dag.teardown() @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): - monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") - a = MockedWorker.remote() - b = MockedWorker.remote() - c = MockedWorker.remote() + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) - ray.get([a.start_mock.remote(), b.start_mock.remote(), c.start_mock.remote()]) + a = Worker.remote() + b = Worker.remote() + c = Worker.remote() with InputNode() as inp: branch1 = a.no_op.bind(inp) @@ -258,8 +269,14 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): assert operation.idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] - ref = compiled_dag.execute(123) - assert ray.get(ref) == [123, 123, 123] + tensor_cpu = torch.zeros(10, 10) + ref = compiled_dag.execute(tensor_cpu) + tensors = ray.get(ref) + tensor_cuda = tensor_cpu.to("cuda:0") + + assert len(tensors) == 3 + for t in tensors: + assert torch.equal(t, tensor_cuda) compiled_dag.teardown() From 2fa6118e0bf478c6884397d7d2d6454b69a444ad Mon Sep 17 00:00:00 2001 From: kaihsun Date: Fri, 2 Aug 2024 23:48:01 +0000 Subject: [PATCH 027/111] add complex test Signed-off-by: kaihsun --- .../experimental/test_execution_schedule.py | 196 +++++++++++++++++- 1 file changed, 195 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index a500c4a5770a1..1c230900af659 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -11,6 +11,10 @@ from ray.dag import InputNode, MultiOutputNode from ray.dag.compiled_dag_node import DAGNodeOperationType import torch +from typing import List +from dataclasses import dataclass, field +from collections import deque, defaultdict +from ray.actor import ActorHandle if sys.platform != "linux" and sys.platform != "darwin": pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) @@ -18,6 +22,170 @@ USE_GPU = bool(os.environ.get("RAY_PYTEST_USE_GPU", 0)) +@dataclass +class PipelineConfig: + pp_size: int + num_micro_batches: int + + +@dataclass +class PipelineUnit: + op: str + pp_rank: int + batch_id: int + uid: str = field(init=False, repr=False) + + def __post_init__(self): + self.uid = f"{self.op}_rank-{self.pp_rank}_batch-{self.batch_id}" + + def __repr__(self) -> str: + return self.uid + + +def generate_1f1b_schedule(config) -> List[List[PipelineUnit]]: + pp_size = config.pp_size + num_micro_batches = config.num_micro_batches + + schedule = [] + for pp_rank in range(config.pp_size): + warm_up_batches = pp_size - pp_rank + main_1f1b_batches = num_micro_batches - warm_up_batches + cool_down_batches = num_micro_batches - main_1f1b_batches + + rank_schedule = [] + bwd_batch_id = fwd_batch_id = 0 + + for _ in range(warm_up_batches): + rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) + fwd_batch_id += 1 + + for _ in range(main_1f1b_batches): + rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) + bwd_batch_id += 1 + rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) + fwd_batch_id += 1 + + for _ in range(cool_down_batches): + rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) + bwd_batch_id += 1 + schedule.append(rank_schedule) + return schedule + + +class PipelineModel: + def __init__( + self, + config: PipelineConfig, + schedule: List[List[PipelineUnit]], + blocks: List[ActorHandle], + compile_dag: bool = True, + ) -> None: + self.config = config + self.blocks = blocks + self.generate_pipeline_schedules(schedule) + self.compile_dag = compile_dag + self.dag = self.build_dag() + + def generate_pipeline_schedules(self, schedule): + self.id_to_unit = dict() + self.stage_schedules = defaultdict(list) + self.batch_schedules = defaultdict(list) + + for pp_rank, stage_schedule in enumerate(schedule): + self.stage_schedules[pp_rank] = stage_schedule + for unit in stage_schedule: + self.id_to_unit[unit.uid] = unit + self.batch_schedules[unit.batch_id].append(unit) + + for batch_id in self.batch_schedules: + fwd_units = [ + unit for unit in self.batch_schedules[batch_id] if unit.op == "FWD" + ] + bwd_units = [ + unit for unit in self.batch_schedules[batch_id] if unit.op == "BWD" + ] + + fwd_units.sort(key=lambda unit: unit.pp_rank) + bwd_units.sort(key=lambda unit: unit.pp_rank, reverse=True) + self.batch_schedules[batch_id] = fwd_units + bwd_units + + def build_dependency_graph(self): + graph = defaultdict(set) + reversed_graph = defaultdict(set) + + for schedules in [self.batch_schedules, self.stage_schedules]: + for schedule in schedules.values(): + prev_unit = None + for unit in schedule: + if prev_unit: + graph[prev_unit.uid].add(unit.uid) + reversed_graph[unit.uid].add(prev_unit.uid) + prev_unit = unit + return graph, reversed_graph + + def build_dag(self): + graph, reversed_graph = self.build_dependency_graph() + dag_nodes = dict() # Cache DAG Node for each unit + + first_unit = self.batch_schedules[0][0] + queue = deque([first_unit.uid]) + + with InputNode() as input_node: + root_node = self.blocks[0].read_input.bind(input_node) + + output_nodes = [] + + while queue: + uid = queue.popleft() + unit = self.id_to_unit[uid] + batch_id = unit.batch_id + batch_schedule_index = self.batch_schedules[batch_id].index(unit) + + # First forward step + if batch_schedule_index == 0: + prev_dag_node = root_node + else: + prev_unit = self.batch_schedules[batch_id][batch_schedule_index - 1] + prev_dag_node = dag_nodes[prev_unit.uid] + + block = self.blocks[unit.pp_rank] + if unit.op == "FWD": + cur_dag_node = block.fwd.bind(prev_dag_node) + else: + cur_dag_node = block.bwd.bind(prev_dag_node) + + # Last backward step + if batch_schedule_index == 2 * self.config.pp_size - 1: + output_nodes.append(cur_dag_node) + + # ADD NCCL Channel: + if unit.op == "FWD" and unit.pp_rank < self.config.pp_size - 1: + cur_dag_node.with_type_hint( + TorchTensorType(transport=TorchTensorType.NCCL) + ) + if unit.op == "BWD" and unit.pp_rank > 0: + cur_dag_node.with_type_hint( + TorchTensorType(transport=TorchTensorType.NCCL) + ) + + dag_nodes[uid] = cur_dag_node + + # Enqueue new units + for target_uid in graph[uid]: + reversed_graph[target_uid].remove(uid) + if not reversed_graph[target_uid]: + queue.append(target_uid) + + dag = MultiOutputNode(output_nodes) + + if self.compile_dag: + dag = dag.experimental_compile() + return dag + + def step(self, input_batches): + return ray.get(self.dag.execute(input_batches)) + + @ray.remote(num_cpus=0, num_gpus=1) class Worker: def __init__(self): @@ -40,7 +208,7 @@ def no_op_two(self, value1, value2): @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) -def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): +def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): """ This test simulates a simple 1F1B pipeline parallelism for training with 2 workers and 2 batches. @@ -135,6 +303,32 @@ def test_simulate_pp_2workers_1f1b(ray_start_regular, monkeypatch): compiled_dag.teardown() +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 4}], indirect=True) +def test_simulate_pp_4workers_8batches_1f1b(ray_start_regular, monkeypatch): + """ + This test simulates a 1F1B pipeline parallelism for training with + 4 workers and 8 batches. + """ + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) + + num_worker, num_batch = 4, 8 + + workers = [Worker.remote() for _ in range(num_worker)] + config = PipelineConfig(num_worker, num_batch) + schedule = generate_1f1b_schedule(config) + model = PipelineModel(config, schedule, workers) + + tensor_cpu = torch.zeros(10, 10) + tensors = model.step(tensor_cpu) + tensor_cuda = tensor_cpu.to("cuda:0") + assert len(tensors) == num_batch + for t in tensors: + assert torch.equal(t, tensor_cuda) + + @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) def test_three_actors_with_nccl_1(ray_start_regular): """ From 75539cab69c15187436b5a60b07552513717d105 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Mon, 5 Aug 2024 23:13:28 +0000 Subject: [PATCH 028/111] move import heapq to the top-level Signed-off-by: Ubuntu --- python/ray/dag/compiled_dag_node.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 00332b95fc1e2..d005e76179d9c 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -40,6 +40,7 @@ from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy from enum import Enum +import heapq @DeveloperAPI @@ -1250,8 +1251,6 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): graph[downstream_idx][DAGNodeOperationType.READ], ) - import heapq - actor_to_candidates = defaultdict(list) for idx, node_dict in graph.items(): for _, node in node_dict.items(): From b4e4a3855d3c6b88939e9a116ab938bb8cee5576 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 5 Aug 2024 16:16:06 -0700 Subject: [PATCH 029/111] Apply suggestions from code review Co-authored-by: Stephanie Wang Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index d005e76179d9c..7ad138f0da53f 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1134,7 +1134,7 @@ def _build_execution_schedule(self): #1 Divide a DAG node into three GraphNodes: READ, COMPUTE, and WRITE. Each GraphNode has a DAGNodeOperation. - #2 Add edges between READ and COMPUTE, and between COMPUTE and WRITE, which + #2 Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which belong to the same task. #3 Add an edge between COMPUTE with bind_index i and COMPUTE with bind_index i+1 if they belong to the same actor. @@ -1149,10 +1149,10 @@ def _build_execution_schedule(self): `bind_index`. If there are multiple candidate nodes with the smallest `bind_index` of the actors that they belong to, any one of them is acceptable. For the implementation details, we maintain a priority queue - for each actor, where the peek of the priority queue is the node with the + for each actor, where the head of the priority queue is the node with the smallest `bind_index`. #2 If the node is an NCCL write node, select it only if all of its downstream - nodes are also the peeks of their priority queues. + nodes are also the heads of their priority queues. #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are NCCL write nodes. In this case, select the one that is the peek of the priority queue and its downstream nodes, regardless of whether the @@ -1263,7 +1263,7 @@ def _select_next_nodes(): """ Select the next nodes for topological sort. This function may return multiple nodes if they are NCCL nodes. In that case, this function only - removes the NCCL write node, which is also the peek of a priority queue. + removes the NCCL write node, which is also the head of a priority queue. Other nodes will be removed in the following iterations. Additionally, visited_nodes ensures that the same node will not be scheduled more than once. From d220045054a85b00b2c158c7b70196417e380f64 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Mon, 5 Aug 2024 23:18:35 +0000 Subject: [PATCH 030/111] update comments Signed-off-by: Ubuntu --- python/ray/dag/compiled_dag_node.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 7ad138f0da53f..89b8a2e152af9 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1136,9 +1136,9 @@ def _build_execution_schedule(self): GraphNode has a DAGNodeOperation. #2 Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which belong to the same task. - #3 Add an edge between COMPUTE with bind_index i and COMPUTE with bind_index + #3 Add an edge from COMPUTE with bind_index i to COMPUTE with bind_index i+1 if they belong to the same actor. - #4 Add an edge between WRITE of the writer task and READ of the reader task. + #4 Add an edge from WRITE of the writer task to READ of the reader task. Step 2: Topological sort From 474e85232248da87566a811a585dd000a10093a1 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Tue, 6 Aug 2024 00:12:13 +0000 Subject: [PATCH 031/111] use existing buffers Signed-off-by: Ubuntu --- python/ray/dag/compiled_dag_node.py | 12 ++- .../experimental/test_execution_schedule.py | 4 + .../channel/serialization_context.py | 75 +++++++------------ python/ray/tests/test_channel.py | 4 +- 4 files changed, 39 insertions(+), 56 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 89b8a2e152af9..5887043c38757 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -206,12 +206,12 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - if op_type == DAGNodeOperationType.READ: try: res = input_reader.read() - ctx.set_intermediate_result(idx, op_type.value, res) + ctx.set_data(idx, res) except RayChannelError: # Channel closed. Exit the loop. return True elif op_type == DAGNodeOperationType.COMPUTE: - res = ctx.get_intermediate_result(idx, DAGNodeOperationType.READ.value) + res = ctx.get_data(idx) method = getattr(self, task.method_name) try: _process_return_vals(res, return_single_output=False) @@ -220,7 +220,7 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - # Propagate it and skip the actual task. We don't need to wrap the # exception in a RayTaskError here because it has already been wrapped # by the previous task. - ctx.set_intermediate_result(idx, op_type.value, exc) + ctx.set_data(idx, exc) return False resolved_inputs = [] @@ -231,11 +231,9 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - output_val = method(*resolved_inputs, **task.resolved_kwargs) except Exception as exc: output_val = _wrap_exception(exc) - ctx.set_intermediate_result(idx, op_type.value, output_val) + ctx.set_data(idx, output_val) elif op_type == DAGNodeOperationType.WRITE: - output_val = ctx.get_intermediate_result( - idx, DAGNodeOperationType.COMPUTE.value - ) + output_val = ctx.get_data(idx) try: output_writer.write(output_val) except RayChannelError: diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 1c230900af659..6c2f8bf90dbe2 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -185,6 +185,9 @@ def build_dag(self): def step(self, input_batches): return ray.get(self.dag.execute(input_batches)) + def teardown(self): + self.dag.teardown() + @ray.remote(num_cpus=0, num_gpus=1) class Worker: @@ -327,6 +330,7 @@ def test_simulate_pp_4workers_8batches_1f1b(ray_start_regular, monkeypatch): assert len(tensors) == num_batch for t in tensors: assert torch.equal(t, tensor_cuda) + model.teardown() @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) diff --git a/python/ray/experimental/channel/serialization_context.py b/python/ray/experimental/channel/serialization_context.py index b064760f7ed53..b9a06e36a3331 100644 --- a/python/ray/experimental/channel/serialization_context.py +++ b/python/ray/experimental/channel/serialization_context.py @@ -9,68 +9,49 @@ class _SerializationContext: def __init__(self): self.use_external_transport: bool = False self.tensors: List["torch.Tensor"] = [] - # Buffer for transferring data between tasks in the same worker process. - # The key is the channel ID, and the value is the data. We don't use a - # lock when reading/writing the buffer because a DAG node actor will only - # execute one task at a time in `do_exec_tasks`. It will not execute multiple - # Ray tasks on a single actor simultaneously. - self.intra_process_channel_buffers: Dict[str, Any] = {} - # The number of readers for each channel. When the number of readers + # Buffer for transferring data between tasks in the same worker process or + # transferring data between READ, COMPUTE, and WRITE operations of the same + # DAG node. We don't use a lock when reading/writing the buffer because a DAG + # node actor will only execute one task at a time in `do_exec_tasks`. It will + # not execute multiple Ray tasks on a single actor simultaneously. + self.intra_process_buffers: Dict[str, Any] = {} + # The number of readers for each key. When the number of readers # reaches 0, remove the data from the buffer. - self.channel_id_to_num_readers: Dict[str, int] = {} - # The key is the order of the task on the actor, which follows the ascending - # order of bind_index, and the value is a dictionary of intermediate results. - # The dictionary maps the operation type, such as "READ", "COMPUTE", and "WRITE" - # , to the intermediate result. - self.intermediate_results_buffer: Dict[int, Dict[str, Any]] = {} + self.key_to_num_readers: Dict[str, int] = {} def set_use_external_transport(self, use_external_transport: bool) -> None: self.use_external_transport = use_external_transport - def set_intermediate_result(self, idx: int, op_type: str, value: Any) -> None: - if idx not in self.intermediate_results_buffer: - self.intermediate_results_buffer[idx] = {} - self.intermediate_results_buffer[idx][op_type] = value - - def get_intermediate_result(self, idx: int, op_type: str) -> Any: - assert ( - idx in self.intermediate_results_buffer - ), f"Index {idx} does not exist in the buffer." - assert ( - op_type in self.intermediate_results_buffer[idx] - ), f"Operation type {op_type} does not exist in the buffer." - return self.intermediate_results_buffer[idx].pop(op_type) - - def set_data(self, channel_id: str, value: Any, num_readers: int) -> None: + def set_data(self, key, value: Any, num_readers: int = 1) -> None: assert num_readers > 0, "num_readers must be greater than 0." assert ( - channel_id not in self.intra_process_channel_buffers - ), f"Channel {channel_id} already exists in the buffer." + key not in self.intra_process_buffers + ), f"Key {key} already exists in the buffer." assert ( - channel_id not in self.channel_id_to_num_readers - ), f"Channel {channel_id} already exists in the channel_id_to_num_readers." + key not in self.key_to_num_readers + ), f"Key {key} already exists in the key_to_num_readers." - self.intra_process_channel_buffers[channel_id] = value - self.channel_id_to_num_readers[channel_id] = num_readers + self.intra_process_buffers[key] = value + self.key_to_num_readers[key] = num_readers - def get_data(self, channel_id: str) -> Any: + def get_data(self, key: str) -> Any: assert ( - channel_id in self.intra_process_channel_buffers - ), f"Channel {channel_id} does not exist in the buffer." + key in self.intra_process_buffers + ), f"Key {key} does not exist in the buffer." assert ( - channel_id in self.channel_id_to_num_readers - ), f"Channel {channel_id} does not exist in the channel_id_to_num_readers." + key in self.key_to_num_readers + ), f"Key {key} does not exist in the key_to_num_readers." - self.channel_id_to_num_readers[channel_id] -= 1 - if self.channel_id_to_num_readers[channel_id] == 0: + self.key_to_num_readers[key] -= 1 + if self.key_to_num_readers[key] == 0: # All readers have read the data, so we can remove it. - self.channel_id_to_num_readers.pop(channel_id) - return self.intra_process_channel_buffers.pop(channel_id) - return self.intra_process_channel_buffers[channel_id] + self.key_to_num_readers.pop(key) + return self.intra_process_buffers.pop(key) + return self.intra_process_buffers[key] - def reset_data(self, channel_id: str) -> None: - self.intra_process_channel_buffers.pop(channel_id, None) - self.channel_id_to_num_readers.pop(channel_id, None) + def reset_data(self, key: str) -> None: + self.intra_process_buffers.pop(key, None) + self.key_to_num_readers.pop(key, None) def reset_tensors(self, tensors: List["torch.Tensor"]) -> List["torch.Tensor"]: prev_tensors = self.tensors diff --git a/python/ray/tests/test_channel.py b/python/ray/tests/test_channel.py index 4f0c1bc2d284f..76725455e35e6 100644 --- a/python/ray/tests/test_channel.py +++ b/python/ray/tests/test_channel.py @@ -699,7 +699,7 @@ def write(self, value): def get_ctx_buffer_size(self): ctx = ray_channel.ChannelContext.get_current().serialization_context - return len(ctx.intra_process_channel_buffers) + return len(ctx.intra_process_buffers) actor = Actor.remote() channel = ray_channel.IntraProcessChannel(num_readers=1) @@ -753,7 +753,7 @@ def write(self, value): def get_ctx_buffer_size(self): ctx = ray_channel.ChannelContext.get_current().serialization_context - return len(ctx.intra_process_channel_buffers) + return len(ctx.intra_process_buffers) actor = Actor.remote() channel = ray_channel.IntraProcessChannel(num_readers=2) From a5891abeab495a537462495e5588ea7f0148f6ce Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Tue, 6 Aug 2024 02:41:56 +0000 Subject: [PATCH 032/111] add prepare / cancel to ExecutableTask Signed-off-by: Ubuntu --- python/ray/dag/compiled_dag_node.py | 44 +++++++++++++---------------- 1 file changed, 19 insertions(+), 25 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 5887043c38757..1ea8e401b692b 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -131,10 +131,8 @@ def do_exec_tasks( tasks: the executable tasks corresponding to the actor methods. """ try: - self._input_readers = [] - self._output_writers = [] for task in tasks: - _prep_task(self, task) + task.prepare() done = False while True: @@ -152,26 +150,8 @@ def do_exec_tasks( @DeveloperAPI def do_cancel_executable_tasks(self, tasks: List["ExecutableTask"]) -> None: - for idx in range(len(tasks)): - self._input_readers[idx].close() - self._output_writers[idx].close() - - -def _prep_task(self, task: "ExecutableTask") -> None: - """ - Prepare the task for execution. - """ - for typ_hint in task.input_type_hints: - typ_hint.register_custom_serializer() - task.output_type_hint.register_custom_serializer() - - input_reader: ReaderInterface = SynchronousReader(task.input_channels) - output_writer: WriterInterface = SynchronousWriter(task.output_channel) - self._input_readers.append(input_reader) - self._output_writers.append(output_writer) - - input_reader.start() - output_writer.start() + for task in tasks: + task.cancel() def _wrap_exception(exc): @@ -199,8 +179,8 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - idx = operation.idx op_type = operation.type - input_reader = self._input_readers[idx] - output_writer = self._output_writers[idx] + input_reader = task._input_reader + output_writer = task._output_writer ctx = ChannelContext.get_current().serialization_context if op_type == DAGNodeOperationType.READ: @@ -429,6 +409,20 @@ def __init__( assert not isinstance(val, ChannelInterface) assert not isinstance(val, DAGInputAdapter) + self._input_reader: ReaderInterface = SynchronousReader(self.input_channels) + self._output_writer: WriterInterface = SynchronousWriter(self.output_channel) + + def cancel(self): + self._input_reader.close() + self._output_writer.close() + + def prepare(self): + for typ_hint in self.input_type_hints: + typ_hint.register_custom_serializer() + self.output_type_hint.register_custom_serializer() + self._input_reader.start() + self._output_writer.start() + @DeveloperAPI class CompiledDAG: From f8112c0c6d6bef31c92d2a3289a53c9d20fe74dc Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 06:41:41 +0000 Subject: [PATCH 033/111] add cache to ExecutableTask Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 41 ++++++++++++++++------------- 1 file changed, 22 insertions(+), 19 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 1ea8e401b692b..8e818f2ba5d74 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -25,7 +25,6 @@ SynchronousWriter, AwaitableBackgroundReader, AwaitableBackgroundWriter, - ChannelContext, ) from ray.util.annotations import DeveloperAPI @@ -176,22 +175,17 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - Returns: True if we are done executing all operations of this actor, False otherwise. """ - idx = operation.idx op_type = operation.type - input_reader = task._input_reader - output_writer = task._output_writer - ctx = ChannelContext.get_current().serialization_context - if op_type == DAGNodeOperationType.READ: try: - res = input_reader.read() - ctx.set_data(idx, res) + res = task.input_reader.read() + task.set_cache(res) except RayChannelError: # Channel closed. Exit the loop. return True elif op_type == DAGNodeOperationType.COMPUTE: - res = ctx.get_data(idx) + res = task.reset_cache() method = getattr(self, task.method_name) try: _process_return_vals(res, return_single_output=False) @@ -200,7 +194,7 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - # Propagate it and skip the actual task. We don't need to wrap the # exception in a RayTaskError here because it has already been wrapped # by the previous task. - ctx.set_data(idx, exc) + task.set_cache(exc) return False resolved_inputs = [] @@ -211,11 +205,11 @@ def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) - output_val = method(*resolved_inputs, **task.resolved_kwargs) except Exception as exc: output_val = _wrap_exception(exc) - ctx.set_data(idx, output_val) + task.set_cache(output_val) elif op_type == DAGNodeOperationType.WRITE: - output_val = ctx.get_data(idx) + output_val = task.reset_cache() try: - output_writer.write(output_val) + task.output_writer.write(output_val) except RayChannelError: # Channel closed. Exit the loop. return True @@ -409,19 +403,28 @@ def __init__( assert not isinstance(val, ChannelInterface) assert not isinstance(val, DAGInputAdapter) - self._input_reader: ReaderInterface = SynchronousReader(self.input_channels) - self._output_writer: WriterInterface = SynchronousWriter(self.output_channel) + self.input_reader: ReaderInterface = SynchronousReader(self.input_channels) + self.output_writer: WriterInterface = SynchronousWriter(self.output_channel) + self._cache = None def cancel(self): - self._input_reader.close() - self._output_writer.close() + self.input_reader.close() + self.output_writer.close() def prepare(self): for typ_hint in self.input_type_hints: typ_hint.register_custom_serializer() self.output_type_hint.register_custom_serializer() - self._input_reader.start() - self._output_writer.start() + self.input_reader.start() + self.output_writer.start() + + def set_cache(self, data): + self._cache = data + + def reset_cache(self): + data = self._cache + self._cache = None + return data @DeveloperAPI From ec43c663c9ed0ca0e85c818679bf61c5f28447f2 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 07:21:13 +0000 Subject: [PATCH 034/111] use ExecutableTask's exec_operation instead Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 103 ++++++++++++++-------------- 1 file changed, 51 insertions(+), 52 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 8e818f2ba5d74..6b1b3d2cb7c79 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -138,8 +138,7 @@ def do_exec_tasks( if done: break for operation in schedule: - task = tasks[operation.idx] - done = _exec_operation(self, task, operation) + done = tasks[operation.idx].exec_operation(self, operation.type) if done: break except Exception: @@ -166,56 +165,6 @@ def _wrap_exception(exc): return wrapped -def _exec_operation(self, task: "ExecutableTask", operation: DAGNodeOperation) -> bool: - """ - Execute the `operation` which belongs to `task`. - Args: - task: The task to execute. - operation: The operation to execute. - Returns: - True if we are done executing all operations of this actor, False otherwise. - """ - op_type = operation.type - - if op_type == DAGNodeOperationType.READ: - try: - res = task.input_reader.read() - task.set_cache(res) - except RayChannelError: - # Channel closed. Exit the loop. - return True - elif op_type == DAGNodeOperationType.COMPUTE: - res = task.reset_cache() - method = getattr(self, task.method_name) - try: - _process_return_vals(res, return_single_output=False) - except Exception as exc: - # Previous task raised an application-level exception. - # Propagate it and skip the actual task. We don't need to wrap the - # exception in a RayTaskError here because it has already been wrapped - # by the previous task. - task.set_cache(exc) - return False - - resolved_inputs = [] - for task_input in task.task_inputs: - resolved_inputs.append(task_input.resolve(res)) - - try: - output_val = method(*resolved_inputs, **task.resolved_kwargs) - except Exception as exc: - output_val = _wrap_exception(exc) - task.set_cache(output_val) - elif op_type == DAGNodeOperationType.WRITE: - output_val = task.reset_cache() - try: - task.output_writer.write(output_val) - except RayChannelError: - # Channel closed. Exit the loop. - return True - return False - - @DeveloperAPI class CompiledTask: """Wraps the normal Ray DAGNode with some metadata.""" @@ -426,6 +375,56 @@ def reset_cache(self): self._cache = None return data + def _read(self): + try: + res = self.input_reader.read() + self.set_cache(res) + return False + except RayChannelError: + # Channel closed. Exit the loop. + return True + + def _compute(self, actor_handle): + res = self.reset_cache() + method = getattr(actor_handle, self.method_name) + try: + _process_return_vals(res, return_single_output=False) + except Exception as exc: + # Previous task raised an application-level exception. + # Propagate it and skip the actual task. We don't need to wrap the + # exception in a RayTaskError here because it has already been wrapped + # by the previous task. + self.set_cache(exc) + return False + + resolved_inputs = [] + for task_input in self.task_inputs: + resolved_inputs.append(task_input.resolve(res)) + + try: + output_val = method(*resolved_inputs, **self.resolved_kwargs) + except Exception as exc: + output_val = _wrap_exception(exc) + self.set_cache(output_val) + return False + + def _write(self): + output_val = self.reset_cache() + try: + self.output_writer.write(output_val) + return False + except RayChannelError: + # Channel closed. Exit the loop. + return True + + def exec_operation(self, actor_handle, op_type): + if op_type == DAGNodeOperationType.READ: + return self._read() + elif op_type == DAGNodeOperationType.COMPUTE: + return self._compute(actor_handle) + elif op_type == DAGNodeOperationType.WRITE: + return self._write() + @DeveloperAPI class CompiledDAG: From 97d45f7e7f49d4e3622795959baf4aac3b10496f Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 15:53:00 +0000 Subject: [PATCH 035/111] restore ser ctx Signed-off-by: Kai-Hsun Chen --- .../channel/serialization_context.py | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/python/ray/experimental/channel/serialization_context.py b/python/ray/experimental/channel/serialization_context.py index b9a06e36a3331..92599a0f8ee86 100644 --- a/python/ray/experimental/channel/serialization_context.py +++ b/python/ray/experimental/channel/serialization_context.py @@ -9,49 +9,49 @@ class _SerializationContext: def __init__(self): self.use_external_transport: bool = False self.tensors: List["torch.Tensor"] = [] - # Buffer for transferring data between tasks in the same worker process or - # transferring data between READ, COMPUTE, and WRITE operations of the same - # DAG node. We don't use a lock when reading/writing the buffer because a DAG - # node actor will only execute one task at a time in `do_exec_tasks`. It will - # not execute multiple Ray tasks on a single actor simultaneously. - self.intra_process_buffers: Dict[str, Any] = {} - # The number of readers for each key. When the number of readers + # Buffer for transferring data between tasks in the same worker process. + # The key is the channel ID, and the value is the data. We don't use a + # lock when reading/writing the buffer because a DAG node actor will only + # execute one task at a time in `do_exec_tasks`. It will not execute multiple + # Ray tasks on a single actor simultaneously. + self.intra_process_channel_buffers: Dict[str, Any] = {} + # The number of readers for each channel. When the number of readers # reaches 0, remove the data from the buffer. - self.key_to_num_readers: Dict[str, int] = {} + self.channel_id_to_num_readers: Dict[str, int] = {} def set_use_external_transport(self, use_external_transport: bool) -> None: self.use_external_transport = use_external_transport - def set_data(self, key, value: Any, num_readers: int = 1) -> None: + def set_data(self, channel_id: str, value: Any, num_readers: int) -> None: assert num_readers > 0, "num_readers must be greater than 0." assert ( - key not in self.intra_process_buffers - ), f"Key {key} already exists in the buffer." + channel_id not in self.intra_process_channel_buffers + ), f"Channel {channel_id} already exists in the buffer." assert ( - key not in self.key_to_num_readers - ), f"Key {key} already exists in the key_to_num_readers." + channel_id not in self.channel_id_to_num_readers + ), f"Channel {channel_id} already exists in the channel_id_to_num_readers." - self.intra_process_buffers[key] = value - self.key_to_num_readers[key] = num_readers + self.intra_process_channel_buffers[channel_id] = value + self.channel_id_to_num_readers[channel_id] = num_readers - def get_data(self, key: str) -> Any: + def get_data(self, channel_id: str) -> Any: assert ( - key in self.intra_process_buffers - ), f"Key {key} does not exist in the buffer." + channel_id in self.intra_process_channel_buffers + ), f"Channel {channel_id} does not exist in the buffer." assert ( - key in self.key_to_num_readers - ), f"Key {key} does not exist in the key_to_num_readers." + channel_id in self.channel_id_to_num_readers + ), f"Channel {channel_id} does not exist in the channel_id_to_num_readers." - self.key_to_num_readers[key] -= 1 - if self.key_to_num_readers[key] == 0: + self.channel_id_to_num_readers[channel_id] -= 1 + if self.channel_id_to_num_readers[channel_id] == 0: # All readers have read the data, so we can remove it. - self.key_to_num_readers.pop(key) - return self.intra_process_buffers.pop(key) - return self.intra_process_buffers[key] + self.channel_id_to_num_readers.pop(channel_id) + return self.intra_process_channel_buffers.pop(channel_id) + return self.intra_process_channel_buffers[channel_id] - def reset_data(self, key: str) -> None: - self.intra_process_buffers.pop(key, None) - self.key_to_num_readers.pop(key, None) + def reset_data(self, channel_id: str) -> None: + self.intra_process_channel_buffers.pop(channel_id, None) + self.channel_id_to_num_readers.pop(channel_id, None) def reset_tensors(self, tensors: List["torch.Tensor"]) -> List["torch.Tensor"]: prev_tensors = self.tensors From 69fa0381069e0af8f5e25ca6c17c068189cb84c5 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 16:08:02 +0000 Subject: [PATCH 036/111] add comments Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 6b1b3d2cb7c79..b620f607516af 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1146,14 +1146,19 @@ def _build_execution_schedule(self): for each actor, where the head of the priority queue is the node with the smallest `bind_index`. #2 If the node is an NCCL write node, select it only if all of its downstream - nodes are also the heads of their priority queues. + nodes are also the heads of their priority queues. That is, the NCCL write + node and all its NCCL read nodes are selected simultaneously. #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are - NCCL write nodes. In this case, select the one that is the peek of the + NCCL write nodes. In this case, select the one that is the head of the priority queue and its downstream nodes, regardless of whether the - downstream nodes are peeks of their priority queues or not. + downstream nodes are heads of their priority queues or not. Then, put the selected nodes into the corresponding actors' schedules. + The goal of the above rules is to build a schedule with fewer bubbles. The + schedule should be intuitive to users, meaning that the execution should + perform operations in ascending order of `bind_index` as much as possible. + [Example]: See `test_execution_schedule` for more examples. From ad51a680fcf936d559129e09e9defa1a46383cc9 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 16:37:32 +0000 Subject: [PATCH 037/111] move DAGNodeOperation to a separate file Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 101 ++++-------------- python/ray/dag/dag_node_operation.py | 71 ++++++++++++ .../experimental/test_execution_schedule.py | 2 +- 3 files changed, 91 insertions(+), 83 deletions(-) create mode 100644 python/ray/dag/dag_node_operation.py diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index b620f607516af..c094534abf0ba 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -37,44 +37,16 @@ _destroy_nccl_group, ) +from ray.dag.dag_node_operation import ( + DAGNodeOperation, + DAGNodeOperationType, + DAGOperationGraphNode, +) + from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy -from enum import Enum import heapq -@DeveloperAPI -class DAGNodeOperationType(Enum): - """ - There are three types of operations that a DAG node can perform: - 1. READ: Read from an input channel. - 2. COMPUTE: Execute the method corresponding to the node. - 3. WRITE: Write to an output channel. - """ - - READ = "READ" - COMPUTE = "COMPUTE" - WRITE = "WRITE" - - -@DeveloperAPI -class DAGNodeOperation: - def __init__( - self, - idx: int, - operation_type: DAGNodeOperationType, - ): - """ - Args: - idx: The index of the task that this operation belongs to - in the actor's ExecutableTask list. The index is not - the same as bind_index, but there are positive correlations - between the two. - operation_type: The type of operation to perform. - """ - self.idx = idx - self.type = operation_type - - # Holds the input arguments for an accelerated DAG node. @PublicAPI(stability="alpha") class RayDAGArgs(NamedTuple): @@ -1126,8 +1098,8 @@ def _build_execution_schedule(self): Step 1: Generate a graph based on the following rules: - #1 Divide a DAG node into three GraphNodes: READ, COMPUTE, and WRITE. Each - GraphNode has a DAGNodeOperation. + #1 Divide a DAG node into three DAGOperationGraphNodes: READ, COMPUTE, + and WRITE. Each DAGOperationGraphNode has a DAGNodeOperation. #2 Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which belong to the same task. #3 Add an edge from COMPUTE with bind_index i to COMPUTE with bind_index @@ -1136,8 +1108,8 @@ def _build_execution_schedule(self): Step 2: Topological sort - If there are multiple GraphNodes with zero in-degree, select one based on - the following rules: + If there are multiple DAGOperationGraphNodes with zero in-degree, select + one based on the following rules: #1 If the nodes are not NCCL write nodes, select the one with the smallest `bind_index`. If there are multiple candidate nodes with the smallest @@ -1167,41 +1139,7 @@ def _build_execution_schedule(self): assert self.actor_to_executable_tasks assert not self.actor_to_execution_schedule - from functools import total_ordering - - @total_ordering - class GraphNode: - def __init__(self, operation: DAGNodeOperation, idx, dag_node): - self.operation = operation - self.idx = idx - assert isinstance(dag_node, ClassMethodNode) - self.actor_handle = dag_node._get_actor_handle() - self.requires_nccl = dag_node.type_hint.requires_nccl() - self.in_edges = set() - self.out_edges = set() - - @property - def in_degree(self) -> int: - return len(self.in_edges) - - def __lt__(self, other): - assert self.actor_handle == other.actor_handle - return self.operation.idx < other.operation.idx - - def __eq__(self, other): - assert self.actor_handle == other.actor_handle - if self.operation.idx == other.operation.idx: - return self.operation.type == other.operation.type - return False - - def __hash__(self): - return hash((self.operation, self.idx)) - - def _add_edge(in_node: GraphNode, out_node: GraphNode): - in_node.out_edges.add((out_node.idx, out_node.operation.type)) - out_node.in_edges.add((in_node.idx, in_node.operation.type)) - - graph: Dict[int, Dict[DAGNodeOperationType, GraphNode]] = {} + graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]] = {} from ray.dag import ( ClassMethodNode, @@ -1212,25 +1150,25 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): prev_compute_node = None for local_idx, exec_task in enumerate(executable_tasks): idx = exec_task.idx - read_node = GraphNode( + read_node = DAGOperationGraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.READ), idx, self.idx_to_task[idx].dag_node, ) - compute_node = GraphNode( + compute_node = DAGOperationGraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.COMPUTE), idx, self.idx_to_task[idx].dag_node, ) - write_node = GraphNode( + write_node = DAGOperationGraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.WRITE), idx, self.idx_to_task[idx].dag_node, ) - _add_edge(read_node, compute_node) - _add_edge(compute_node, write_node) + read_node.add_edge(compute_node) + compute_node.add_edge(write_node) if prev_compute_node is not None: - _add_edge(prev_compute_node, compute_node) + prev_compute_node.add_edge(compute_node) prev_compute_node = compute_node graph[idx] = { DAGNodeOperationType.READ: read_node, @@ -1245,9 +1183,8 @@ def _add_edge(in_node: GraphNode, out_node: GraphNode): downstream_dag_node = self.idx_to_task[downstream_idx].dag_node if isinstance(downstream_dag_node, MultiOutputNode): continue - _add_edge( - graph[idx][DAGNodeOperationType.WRITE], - graph[downstream_idx][DAGNodeOperationType.READ], + graph[idx][DAGNodeOperationType.WRITE].add_edge( + graph[downstream_idx][DAGNodeOperationType.READ] ) actor_to_candidates = defaultdict(list) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py new file mode 100644 index 0000000000000..9e8de0e9bb4ec --- /dev/null +++ b/python/ray/dag/dag_node_operation.py @@ -0,0 +1,71 @@ +from functools import total_ordering +from enum import Enum +from ray.util.annotations import DeveloperAPI + + +@DeveloperAPI +class DAGNodeOperationType(Enum): + """ + There are three types of operations that a DAG node can perform: + 1. READ: Read from an input channel. + 2. COMPUTE: Execute the method corresponding to the node. + 3. WRITE: Write to an output channel. + """ + + READ = "READ" + COMPUTE = "COMPUTE" + WRITE = "WRITE" + + +@DeveloperAPI +class DAGNodeOperation: + def __init__( + self, + idx: int, + operation_type: DAGNodeOperationType, + ): + """ + Args: + idx: The index of the task that this operation belongs to + in the actor's ExecutableTask list. The index is not + the same as bind_index, but there are positive correlations + between the two. + operation_type: The type of operation to perform. + """ + self.idx = idx + self.type = operation_type + + +@total_ordering +class DAGOperationGraphNode: + def __init__(self, operation: DAGNodeOperation, idx, dag_node): + self.operation = operation + self.idx = idx + from ray.dag import ClassMethodNode + + assert isinstance(dag_node, ClassMethodNode) + self.actor_handle = dag_node._get_actor_handle() + self.requires_nccl = dag_node.type_hint.requires_nccl() + self.in_edges = set() + self.out_edges = set() + + @property + def in_degree(self) -> int: + return len(self.in_edges) + + def __lt__(self, other): + assert self.actor_handle == other.actor_handle + return self.operation.idx < other.operation.idx + + def __eq__(self, other): + assert self.actor_handle == other.actor_handle + if self.operation.idx == other.operation.idx: + return self.operation.type == other.operation.type + return False + + def __hash__(self): + return hash((self.operation, self.idx)) + + def add_edge(self, out_node: "DAGOperationGraphNode"): + self.out_edges.add((out_node.idx, out_node.operation.type)) + out_node.in_edges.add((self.idx, self.operation.type)) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 6c2f8bf90dbe2..a5fddca76738a 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -9,7 +9,7 @@ from ray.experimental.channel.torch_tensor_type import TorchTensorType from ray.tests.conftest import * # noqa from ray.dag import InputNode, MultiOutputNode -from ray.dag.compiled_dag_node import DAGNodeOperationType +from ray.dag.dag_node_operation import DAGNodeOperationType import torch from typing import List from dataclasses import dataclass, field From ef28fd73dc5cec64b42df39b5039d5d5696855b3 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 16:49:06 +0000 Subject: [PATCH 038/111] add comments to DAGOperationGraphNode Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 9e8de0e9bb4ec..72d250bee2967 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -54,10 +54,20 @@ def in_degree(self) -> int: return len(self.in_edges) def __lt__(self, other): + """ + Two DAGOperationGraphNodes are comparable only when they belong to + the same actor. For operations on the same actor, if idx is smaller, + the DAGNode to which this operation belongs has a smaller `bind_index`. + """ assert self.actor_handle == other.actor_handle return self.operation.idx < other.operation.idx def __eq__(self, other): + """ + Two DAGOperationGraphNodes are comparable only when they belong to the + same actor. For operations on the same actor, two operations are equal + only when they have the same `idx` and `type`. + """ assert self.actor_handle == other.actor_handle if self.operation.idx == other.operation.idx: return self.operation.type == other.operation.type From acb78f436123cf7dda8e4cd293d53eb4b96e8f2a Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 16:57:18 +0000 Subject: [PATCH 039/111] add comments for building graph Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index c094534abf0ba..43ee126bc76b3 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1146,9 +1146,12 @@ def _build_execution_schedule(self): MultiOutputNode, ) + # Step 1: Build a graph for _, executable_tasks in self.actor_to_executable_tasks.items(): prev_compute_node = None for local_idx, exec_task in enumerate(executable_tasks): + # Divide a DAG node into three DAGOperationGraphNodes: READ, COMPUTE, + # and WRITE. Each DAGOperationGraphNode has a DAGNodeOperation. idx = exec_task.idx read_node = DAGOperationGraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.READ), @@ -1165,8 +1168,12 @@ def _build_execution_schedule(self): idx, self.idx_to_task[idx].dag_node, ) + # Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which + # belong to the same task. read_node.add_edge(compute_node) compute_node.add_edge(write_node) + # Add an edge from COMPUTE with `bind_index` i to COMPUTE with + # `bind_index` i+1 if they belong to the same actor. if prev_compute_node is not None: prev_compute_node.add_edge(compute_node) prev_compute_node = compute_node @@ -1176,6 +1183,7 @@ def _build_execution_schedule(self): DAGNodeOperationType.WRITE: write_node, } + # Add an edge from WRITE of the writer task to READ of the reader task. for idx, task in self.idx_to_task.items(): if not isinstance(task.dag_node, ClassMethodNode): continue @@ -1248,6 +1256,7 @@ def _select_next_nodes(): next_nodes.append(downstream_node) return next_nodes + # Step 2: Topological sort while actor_to_candidates: nodes = _select_next_nodes() for node in nodes: From e9ecbe5cc59322c0cbce184338fcd29e51af804b Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 18:00:19 +0000 Subject: [PATCH 040/111] fix Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 1 + python/ray/tests/test_channel.py | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 72d250bee2967..c6bdf9cf2b481 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -36,6 +36,7 @@ def __init__( self.type = operation_type +@DeveloperAPI @total_ordering class DAGOperationGraphNode: def __init__(self, operation: DAGNodeOperation, idx, dag_node): diff --git a/python/ray/tests/test_channel.py b/python/ray/tests/test_channel.py index 76725455e35e6..4f0c1bc2d284f 100644 --- a/python/ray/tests/test_channel.py +++ b/python/ray/tests/test_channel.py @@ -699,7 +699,7 @@ def write(self, value): def get_ctx_buffer_size(self): ctx = ray_channel.ChannelContext.get_current().serialization_context - return len(ctx.intra_process_buffers) + return len(ctx.intra_process_channel_buffers) actor = Actor.remote() channel = ray_channel.IntraProcessChannel(num_readers=1) @@ -753,7 +753,7 @@ def write(self, value): def get_ctx_buffer_size(self): ctx = ray_channel.ChannelContext.get_current().serialization_context - return len(ctx.intra_process_buffers) + return len(ctx.intra_process_channel_buffers) actor = Actor.remote() channel = ray_channel.IntraProcessChannel(num_readers=2) From bf6957060d2209d1bd9c827f56b5f45524f13541 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 18:55:57 +0000 Subject: [PATCH 041/111] move select_next_nodes to top-level Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 106 ++++++++++++++-------------- 1 file changed, 52 insertions(+), 54 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 43ee126bc76b3..17f066a2d988a 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -137,6 +137,57 @@ def _wrap_exception(exc): return wrapped +def _select_next_nodes(actor_to_candidates, graph): + """ + Select the next nodes for topological sort. This function may return + multiple nodes if they are NCCL nodes. In that case, this function only + removes the NCCL write node, which is also the head of a priority queue. + Other nodes will be removed in the following iterations. Additionally, + visited_nodes ensures that the same node will not be scheduled more than + once. + """ + next_nodes = [] + first_nccl_node = None + for _, candidates in actor_to_candidates.items(): + if ( + not candidates[0].requires_nccl + or candidates[0].operation.type != DAGNodeOperationType.WRITE + ): + next_nodes.append(heapq.heappop(candidates)) + return next_nodes + if first_nccl_node is None: + first_nccl_node = candidates[0] + is_next_node = True + for downstream_node_metadata in candidates[0].out_edges: + downstream_node = graph[downstream_node_metadata[0]][ + downstream_node_metadata[1] + ] + downstream_node_actor = downstream_node.actor_handle + if ( + downstream_node_actor not in actor_to_candidates + or downstream_node != actor_to_candidates[downstream_node_actor][0] + ): + is_next_node = False + break + if is_next_node: + next_nodes.append(heapq.heappop(candidates)) + for downstream_node_metadata in next_nodes[0].out_edges: + downstream_node = graph[downstream_node_metadata[0]][ + downstream_node_metadata[1] + ] + next_nodes.append(downstream_node) + return next_nodes + + assert first_nccl_node is not None + next_nodes.append(heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle])) + for downstream_node_metadata in first_nccl_node.out_edges: + downstream_node = graph[downstream_node_metadata[0]][ + downstream_node_metadata[1] + ] + next_nodes.append(downstream_node) + return next_nodes + + @DeveloperAPI class CompiledTask: """Wraps the normal Ray DAGNode with some metadata.""" @@ -1203,62 +1254,9 @@ def _build_execution_schedule(self): visited_nodes = set() - def _select_next_nodes(): - """ - Select the next nodes for topological sort. This function may return - multiple nodes if they are NCCL nodes. In that case, this function only - removes the NCCL write node, which is also the head of a priority queue. - Other nodes will be removed in the following iterations. Additionally, - visited_nodes ensures that the same node will not be scheduled more than - once. - """ - next_nodes = [] - first_nccl_node = None - for _, candidates in actor_to_candidates.items(): - if ( - not candidates[0].requires_nccl - or candidates[0].operation.type != DAGNodeOperationType.WRITE - ): - next_nodes.append(heapq.heappop(candidates)) - return next_nodes - if first_nccl_node is None: - first_nccl_node = candidates[0] - is_next_node = True - for downstream_node_metadata in candidates[0].out_edges: - downstream_node = graph[downstream_node_metadata[0]][ - downstream_node_metadata[1] - ] - downstream_node_actor = downstream_node.actor_handle - if ( - downstream_node_actor not in actor_to_candidates - or downstream_node - != actor_to_candidates[downstream_node_actor][0] - ): - is_next_node = False - break - if is_next_node: - next_nodes.append(heapq.heappop(candidates)) - for downstream_node_metadata in next_nodes[0].out_edges: - downstream_node = graph[downstream_node_metadata[0]][ - downstream_node_metadata[1] - ] - next_nodes.append(downstream_node) - return next_nodes - - assert first_nccl_node is not None - next_nodes.append( - heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle]) - ) - for downstream_node_metadata in first_nccl_node.out_edges: - downstream_node = graph[downstream_node_metadata[0]][ - downstream_node_metadata[1] - ] - next_nodes.append(downstream_node) - return next_nodes - # Step 2: Topological sort while actor_to_candidates: - nodes = _select_next_nodes() + nodes = _select_next_nodes(actor_to_candidates, graph) for node in nodes: if node in visited_nodes: continue From b18df948dafb0059212ba5e7a4cf95bb03e40c1f Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 20:12:13 +0000 Subject: [PATCH 042/111] move comments to select_next_nodes Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 47 +++++++++++++++-------------- 1 file changed, 25 insertions(+), 22 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 17f066a2d988a..8b63f8d770a0a 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -139,12 +139,29 @@ def _wrap_exception(exc): def _select_next_nodes(actor_to_candidates, graph): """ - Select the next nodes for topological sort. This function may return - multiple nodes if they are NCCL nodes. In that case, this function only - removes the NCCL write node, which is also the head of a priority queue. - Other nodes will be removed in the following iterations. Additionally, - visited_nodes ensures that the same node will not be scheduled more than - once. + This function selects the next nodes for topological sort. If there are + multiple DAGOperationGraphNodes with zero in-degree, select nodes based on + the following rules: + + #1 If the nodes are not NCCL write nodes, select the one with the smallest + `bind_index`. If there are multiple candidate nodes with the smallest + `bind_index` of the actors that they belong to, any one of them is + acceptable. For the implementation details, we maintain a priority queue + for each actor, where the head of the priority queue is the node with the + smallest `bind_index`. + #2 If the node is an NCCL write node, select it only if all of its downstream + nodes are also the heads of their priority queues. That is, the NCCL write + node and all its NCCL read nodes are selected simultaneously. + #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are + NCCL write nodes. In this case, select the one that is the head of the + priority queue and its downstream nodes, regardless of whether the + downstream nodes are heads of their priority queues or not. + + This function may return multiple nodes if they are NCCL nodes. In that case, + this function only removes the NCCL write node, which is also the head of a + priority queue. Other nodes will be removed in the following iterations. + Additionally, visited_nodes ensures that the same node will not be scheduled + more than once. """ next_nodes = [] first_nccl_node = None @@ -1159,22 +1176,8 @@ def _build_execution_schedule(self): Step 2: Topological sort - If there are multiple DAGOperationGraphNodes with zero in-degree, select - one based on the following rules: - - #1 If the nodes are not NCCL write nodes, select the one with the smallest - `bind_index`. If there are multiple candidate nodes with the smallest - `bind_index` of the actors that they belong to, any one of them is - acceptable. For the implementation details, we maintain a priority queue - for each actor, where the head of the priority queue is the node with the - smallest `bind_index`. - #2 If the node is an NCCL write node, select it only if all of its downstream - nodes are also the heads of their priority queues. That is, the NCCL write - node and all its NCCL read nodes are selected simultaneously. - #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are - NCCL write nodes. In this case, select the one that is the head of the - priority queue and its downstream nodes, regardless of whether the - downstream nodes are heads of their priority queues or not. + It is possible to have multiple DAGOperationGraphNodes with zero in-degree. + Refer to the function `_select_next_nodes` for the logic of selecting nodes. Then, put the selected nodes into the corresponding actors' schedules. From 35288f6f0ea966236c526123d41f47897d56531d Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 20:30:41 +0000 Subject: [PATCH 043/111] add comments for schedule Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 8b63f8d770a0a..39f37646333e5 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -100,6 +100,7 @@ def do_exec_tasks( Args: tasks: the executable tasks corresponding to the actor methods. + schedule: A list of `DAGNodeOperations` that should be executed in order. """ try: for task in tasks: From ec7f191be7bcf99f3e4b0ebb2a8bf8affb8b92a1 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 21:48:12 +0000 Subject: [PATCH 044/111] add comments for ExecutableTask Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 45 ++++++++++++++++++++++++++--- 1 file changed, 41 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 39f37646333e5..0e10f09a0424e 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -393,8 +393,13 @@ def __init__( assert not isinstance(val, ChannelInterface) assert not isinstance(val, DAGInputAdapter) + # Input reader to read input data from upstream DAG nodes. self.input_reader: ReaderInterface = SynchronousReader(self.input_channels) + # Output writer to write output data to downstream DAG nodes. self.output_writer: WriterInterface = SynchronousWriter(self.output_channel) + # Store the intermediate result of a READ or COMPUTE operation. + # The result of a READ operation will be used by a COMPUTE operation, + # and the result of a COMPUTE operation will be used by a WRITE operation. self._cache = None def cancel(self): @@ -402,6 +407,10 @@ def cancel(self): self.output_writer.close() def prepare(self): + """ + Prepare the task for execution. The `exec_operation` function can only + be called after `prepare` has been called. + """ for typ_hint in self.input_type_hints: typ_hint.register_custom_serializer() self.output_type_hint.register_custom_serializer() @@ -417,6 +426,9 @@ def reset_cache(self): return data def _read(self): + """ + Read input data from upstream DAG nodes and cache the intermediate result. + """ try: res = self.input_reader.read() self.set_cache(res) @@ -425,9 +437,15 @@ def _read(self): # Channel closed. Exit the loop. return True - def _compute(self, actor_handle): + def _compute(self, class_handle): + """ + Retrieve the intermediate result from the READ operation and perform the + computation. Then, cache the new intermediate result. The caller must ensure + that the last operation executed is READ so that the function retrieves the + correct intermediate result. + """ res = self.reset_cache() - method = getattr(actor_handle, self.method_name) + method = getattr(class_handle, self.method_name) try: _process_return_vals(res, return_single_output=False) except Exception as exc: @@ -450,6 +468,11 @@ def _compute(self, actor_handle): return False def _write(self): + """ + Retrieve the intermediate result from the COMPUTE operation and write to its + downstream DAG nodes. The caller must ensure that the last operation executed + is COMPUTE so that the function retrieves the correct intermediate result. + """ output_val = self.reset_cache() try: self.output_writer.write(output_val) @@ -458,11 +481,25 @@ def _write(self): # Channel closed. Exit the loop. return True - def exec_operation(self, actor_handle, op_type): + def exec_operation(self, class_handle, op_type: DAGNodeOperationType): + """ + An ExecutableTask corresponds to a DAGNode. It consists of three + operations: READ, COMPUTE, and WRITE, which should be executed in + order to ensure that each operation can read the correct intermediate + result. + + Args: + class_handle: The handle of the class to which the actor belongs. + op_type: The type of the operation. Possible types are READ, + COMPUTE, and WRITE. + + Returns: + True if the next operation should not be executed; otherwise, False. + """ if op_type == DAGNodeOperationType.READ: return self._read() elif op_type == DAGNodeOperationType.COMPUTE: - return self._compute(actor_handle) + return self._compute(class_handle) elif op_type == DAGNodeOperationType.WRITE: return self._write() From d7ae7a0834ad7b4439ce00295992e9c863f5df1e Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 21:58:40 +0000 Subject: [PATCH 045/111] update select_next_nodes comment Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 0e10f09a0424e..59980f2c60908 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -140,9 +140,9 @@ def _wrap_exception(exc): def _select_next_nodes(actor_to_candidates, graph): """ - This function selects the next nodes for topological sort. If there are - multiple DAGOperationGraphNodes with zero in-degree, select nodes based on - the following rules: + This function selects the next nodes for topological sort to generate execution + schedule. If there are multiple DAGOperationGraphNodes with zero in-degree, + select nodes based on the following rules: #1 If the nodes are not NCCL write nodes, select the one with the smallest `bind_index`. If there are multiple candidate nodes with the smallest From aeaeddaf0e36befab84370fa01b7f0b60e02e948 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 22:34:23 +0000 Subject: [PATCH 046/111] add comments for DAGOperationGraphNode Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 11 ++++++++++- python/ray/dag/dag_node_operation.py | 16 +++++++++++++++- 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 59980f2c60908..b50da44d8fe07 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -138,7 +138,10 @@ def _wrap_exception(exc): return wrapped -def _select_next_nodes(actor_to_candidates, graph): +def _select_next_nodes( + actor_to_candidates: Dict["ray.actor.ActorHandle", List[DAGOperationGraphNode]], + graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], +): """ This function selects the next nodes for topological sort to generate execution schedule. If there are multiple DAGOperationGraphNodes with zero in-degree, @@ -163,6 +166,12 @@ def _select_next_nodes(actor_to_candidates, graph): priority queue. Other nodes will be removed in the following iterations. Additionally, visited_nodes ensures that the same node will not be scheduled more than once. + + Args: + actor_to_candidates: A dictionary mapping an actor handle to a list of + candidate nodes with zero in-degree. + graph: A dictionary mapping the index of a task to a dictionary of its + DAGOperationGraphNodes for different operations. """ next_nodes = [] first_nccl_node = None diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index c6bdf9cf2b481..e588dbd68a596 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -1,6 +1,7 @@ from functools import total_ordering from enum import Enum from ray.util.annotations import DeveloperAPI +import ray @DeveloperAPI @@ -39,7 +40,20 @@ def __init__( @DeveloperAPI @total_ordering class DAGOperationGraphNode: - def __init__(self, operation: DAGNodeOperation, idx, dag_node): + def __init__( + self, operation: DAGNodeOperation, idx: int, dag_node: "ray.dag.DAGNode" + ): + """ + DAGOperationGraphNode represents a node in the DAG operation graph. + It contains information about the node's in-degree, out-degree, edges, + and the operation it performs. + + Args: + operation: The operation that this node performs. The operation + can be a READ, COMPUTE, or WRITE operation. + idx: A unique index into the original DAG. + dag_node: The DAGNode that this operation belongs to. + """ self.operation = operation self.idx = idx from ray.dag import ClassMethodNode From 422a2d98a963ddb4406b530c72127f1b8e07b7d4 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 6 Aug 2024 22:46:59 +0000 Subject: [PATCH 047/111] separate build graph into a independent func Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 51 ++++++++++++++++++----------- 1 file changed, 31 insertions(+), 20 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index b50da44d8fe07..ddea8bf959806 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1206,12 +1206,9 @@ def _get_or_compile( self._dag_submitter.start() self._dag_output_fetcher.start() - def _build_execution_schedule(self): + def _build_dag_node_operation_graph(self): """ - Generate an execution schedule for each actor. The schedule is a list of - DAGNodeOperation. - - Step 1: Generate a graph based on the following rules: + Generate a DAG node operation graph based on the following rules: #1 Divide a DAG node into three DAGOperationGraphNodes: READ, COMPUTE, and WRITE. Each DAGOperationGraphNode has a DAGNodeOperation. @@ -1221,20 +1218,10 @@ def _build_execution_schedule(self): i+1 if they belong to the same actor. #4 Add an edge from WRITE of the writer task to READ of the reader task. - Step 2: Topological sort + This is the step one of building an execution schedule for each actor. - It is possible to have multiple DAGOperationGraphNodes with zero in-degree. - Refer to the function `_select_next_nodes` for the logic of selecting nodes. - - Then, put the selected nodes into the corresponding actors' schedules. - - The goal of the above rules is to build a schedule with fewer bubbles. The - schedule should be intuitive to users, meaning that the execution should - perform operations in ascending order of `bind_index` as much as possible. - - [Example]: - - See `test_execution_schedule` for more examples. + Returns: + A graph that each node is a DAGOperationGraphNode. """ assert self.idx_to_task assert self.actor_to_executable_tasks @@ -1295,6 +1282,32 @@ def _build_execution_schedule(self): graph[idx][DAGNodeOperationType.WRITE].add_edge( graph[downstream_idx][DAGNodeOperationType.READ] ) + return graph + + def _build_execution_schedule(self): + """ + Generate an execution schedule for each actor. The schedule is a list of + DAGNodeOperation. + + Step 1: Generate a DAG node operation graph. Refer to the function + `_build_dag_node_operation_graph` for more details. + + Step 2: Topological sort + + It is possible to have multiple DAGOperationGraphNodes with zero in-degree. + Refer to the function `_select_next_nodes` for the logic of selecting nodes. + + Then, put the selected nodes into the corresponding actors' schedules. + + The goal of the above rules is to build a schedule with fewer bubbles. The + schedule should be intuitive to users, meaning that the execution should + perform operations in ascending order of `bind_index` as much as possible. + + [Example]: + + See `test_execution_schedule` for more examples. + """ + graph = self._build_dag_node_operation_graph() actor_to_candidates = defaultdict(list) for idx, node_dict in graph.items(): @@ -1329,8 +1342,6 @@ def _build_execution_schedule(self): for key in delete_keys: del actor_to_candidates[key] - # TODO: Check whether topological sort exists or not. - def _detect_deadlock(self) -> bool: """ Create a graph with the following 3 rules, and then use From f9896650514ec161f042dba0546da39be6261934 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 00:39:37 +0000 Subject: [PATCH 048/111] add tests for _select_next_nodes Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 40 ++-- python/ray/dag/dag_node_operation.py | 13 +- .../experimental/test_execution_schedule.py | 191 +++++++++++++++++- 3 files changed, 209 insertions(+), 35 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index ddea8bf959806..689fbb80176c2 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -153,10 +153,7 @@ def _select_next_nodes( acceptable. For the implementation details, we maintain a priority queue for each actor, where the head of the priority queue is the node with the smallest `bind_index`. - #2 If the node is an NCCL write node, select it only if all of its downstream - nodes are also the heads of their priority queues. That is, the NCCL write - node and all its NCCL read nodes are selected simultaneously. - #3 If #1 and #2 cannot be satisfied, it means that all candidate nodes are + #2 If #1 cannot be satisfied, it means that all candidate nodes are NCCL write nodes. In this case, select the one that is the head of the priority queue and its downstream nodes, regardless of whether the downstream nodes are heads of their priority queues or not. @@ -184,26 +181,6 @@ def _select_next_nodes( return next_nodes if first_nccl_node is None: first_nccl_node = candidates[0] - is_next_node = True - for downstream_node_metadata in candidates[0].out_edges: - downstream_node = graph[downstream_node_metadata[0]][ - downstream_node_metadata[1] - ] - downstream_node_actor = downstream_node.actor_handle - if ( - downstream_node_actor not in actor_to_candidates - or downstream_node != actor_to_candidates[downstream_node_actor][0] - ): - is_next_node = False - break - if is_next_node: - next_nodes.append(heapq.heappop(candidates)) - for downstream_node_metadata in next_nodes[0].out_edges: - downstream_node = graph[downstream_node_metadata[0]][ - downstream_node_metadata[1] - ] - next_nodes.append(downstream_node) - return next_nodes assert first_nccl_node is not None next_nodes.append(heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle])) @@ -1241,20 +1218,27 @@ def _build_dag_node_operation_graph(self): # Divide a DAG node into three DAGOperationGraphNodes: READ, COMPUTE, # and WRITE. Each DAGOperationGraphNode has a DAGNodeOperation. idx = exec_task.idx + dag_node = self.idx_to_task[idx].dag_node + actor_handle = dag_node._get_actor_handle() + requires_nccl = dag_node.type_hint.requires_nccl() + read_node = DAGOperationGraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.READ), idx, - self.idx_to_task[idx].dag_node, + actor_handle, + requires_nccl, ) compute_node = DAGOperationGraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.COMPUTE), idx, - self.idx_to_task[idx].dag_node, + actor_handle, + requires_nccl, ) write_node = DAGOperationGraphNode( DAGNodeOperation(local_idx, DAGNodeOperationType.WRITE), idx, - self.idx_to_task[idx].dag_node, + actor_handle, + requires_nccl, ) # Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which # belong to the same task. @@ -1310,7 +1294,7 @@ def _build_execution_schedule(self): graph = self._build_dag_node_operation_graph() actor_to_candidates = defaultdict(list) - for idx, node_dict in graph.items(): + for _, node_dict in graph.items(): for _, node in node_dict.items(): if node.in_degree == 0: heapq.heappush(actor_to_candidates[node.actor_handle], node) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index e588dbd68a596..b55696cc92128 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -41,7 +41,11 @@ def __init__( @total_ordering class DAGOperationGraphNode: def __init__( - self, operation: DAGNodeOperation, idx: int, dag_node: "ray.dag.DAGNode" + self, + operation: DAGNodeOperation, + idx: int, + actor_handle: "ray.actor.ActorHandle", + requires_nccl: bool, ): """ DAGOperationGraphNode represents a node in the DAG operation graph. @@ -56,11 +60,8 @@ def __init__( """ self.operation = operation self.idx = idx - from ray.dag import ClassMethodNode - - assert isinstance(dag_node, ClassMethodNode) - self.actor_handle = dag_node._get_actor_handle() - self.requires_nccl = dag_node.type_hint.requires_nccl() + self.actor_handle = actor_handle + self.requires_nccl = requires_nccl self.in_edges = set() self.out_edges = set() diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index a5fddca76738a..5d666df882132 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -9,7 +9,12 @@ from ray.experimental.channel.torch_tensor_type import TorchTensorType from ray.tests.conftest import * # noqa from ray.dag import InputNode, MultiOutputNode -from ray.dag.dag_node_operation import DAGNodeOperationType +from ray.dag.dag_node_operation import ( + DAGNodeOperationType, + DAGOperationGraphNode, + DAGNodeOperation, +) +from ray.dag.compiled_dag_node import _select_next_nodes import torch from typing import List from dataclasses import dataclass, field @@ -479,6 +484,190 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): compiled_dag.teardown() +def generate_dag_graph_nodes(local_idx, global_idx, actor_handle, requires_nccl): + graph_nodes = {} + for op_type in DAGNodeOperationType: + graph_nodes[op_type] = DAGOperationGraphNode( + DAGNodeOperation(local_idx, op_type), + global_idx, + actor_handle, + requires_nccl, + ) + return graph_nodes + + +class TestSelectNextNodes: + """ + Test whether `_select_next_nodes` function selects the next nodes for + topological sort to generate execution schedule correctly. + + global_idx: Each DAG node has a unique global index. + local_idx: The DAG node's index in the actor's `executable_tasks` list. + """ + + def test_two_candidates_on_same_actor(self): + """ + Simulate the case where there are two candidates on the same actor. + The candidate with the smaller index in the `executable_tasks` list + should be selected. + + driver -> fake_actor.op -> fake_actor.op -> driver + + In the example above, both READ operations on the fake_actor have zero + in-degree. The operation with the smaller index in the executable_tasks + list should be selected first; therefore, the one on the left side will + be selected first. + """ + fake_actor = "fake_actor" + # The DAG node has a global index of 1, and its index in the + # actor's `executable_tasks` list is 0. + global_idx_1 = 1 + dag_node_1 = DAGOperationGraphNode( + DAGNodeOperation(0, DAGNodeOperationType.READ), + global_idx_1, + fake_actor, + False, + ) + # The DAG node has a global index of 2, and its index in the + # actor's `executable_tasks` list is 1. + global_idx_2 = 2 + dag_node_2 = DAGOperationGraphNode( + DAGNodeOperation(1, DAGNodeOperationType.READ), + global_idx_2, + fake_actor, + False, + ) + mock_actor_to_candidates = { + fake_actor: [ + dag_node_1, + dag_node_2, + ], + } + next_nodes = _select_next_nodes(mock_actor_to_candidates, None) + assert len(next_nodes) == 1 + assert next_nodes[0] == dag_node_1 + + def test_only_one_nccl_write(self): + """ + Simulate the case where there is only one candidate which is a NCCL + WRITE operation. In this case, `_select_next_nodes` should return both + the NCCL WRITE operation and the corresponding READ operation. + + driver -> fake_actor_1.op -> fake_actor_2.op -> driver + + In the example above, communication between fake_actor_1 and fake_actor_2 + is done using NCCL. The following test case simulates a scenario where the + READ and COMPUTE operations on fake_actor_1 have already been added to the + execution schedule. + """ + fake_actor_1, global_idx_1, local_idx_1 = "fake_actor_1", 1, 0 + fake_actor_2, global_idx_2, local_idx_2 = "fake_actor_2", 2, 0 + mock_graph = { + global_idx_1: generate_dag_graph_nodes( + local_idx_1, global_idx_1, fake_actor_1, True + ), + global_idx_2: generate_dag_graph_nodes( + local_idx_2, global_idx_2, fake_actor_2, False + ), + } + del mock_graph[global_idx_1][DAGNodeOperationType.READ] + del mock_graph[global_idx_1][DAGNodeOperationType.COMPUTE] + mock_graph[global_idx_1][DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_2][DAGNodeOperationType.READ] + ) + mock_graph[global_idx_2][DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_2][DAGNodeOperationType.COMPUTE] + ) + mock_graph[global_idx_2][DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_2][DAGNodeOperationType.WRITE] + ) + mock_actor_to_candidates = { + fake_actor_1: [mock_graph[global_idx_1][DAGNodeOperationType.WRITE]], + } + next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) + assert len(next_nodes) == 2 + assert next_nodes[0] == mock_graph[global_idx_1][DAGNodeOperationType.WRITE] + assert next_nodes[1] == mock_graph[global_idx_2][DAGNodeOperationType.READ] + + def test_two_nccl_writes(self): + """ + Simulate a scenario where there are two candidates that are NCCL WRITE + operations. In this case, _select_next_nodes can choose either of the + two NCCL WRITE operations and their corresponding READ operations. + + driver -> fake_actor_1.op -> fake_actor_2.op -> driver + | | + -> fake_actor_2.op -> fake_actor_1.op - + + In the example above, communication between fake_actor_1 and fake_actor_2 is + done using NCCL. The following test case simulates a scenario where the READ + and COMPUTE operations on both the DAG nodes with smaller bind_index on + fake_actor_1 and fake_actor_2 have already been added to the execution schedule. + """ + fake_actor_1 = "fake_actor_1" + global_idx_1_0, local_idx_1_0 = 1, 0 + global_idx_1_1, local_idx_1_1 = 3, 1 + fake_actor_2 = "fake_actor_2" + global_idx_2_0, local_idx_2_0 = 2, 0 + global_idx_2_1, local_idx_2_1 = 4, 1 + mock_graph = { + global_idx_1_0: generate_dag_graph_nodes( + local_idx_1_0, global_idx_1_0, fake_actor_1, True + ), + global_idx_1_1: generate_dag_graph_nodes( + local_idx_1_1, global_idx_1_1, fake_actor_1, False + ), + global_idx_2_0: generate_dag_graph_nodes( + local_idx_2_0, global_idx_2_0, fake_actor_2, True + ), + global_idx_2_1: generate_dag_graph_nodes( + local_idx_2_1, global_idx_2_1, fake_actor_2, False + ), + } + del mock_graph[global_idx_1_0][DAGNodeOperationType.READ] + del mock_graph[global_idx_1_0][DAGNodeOperationType.COMPUTE] + del mock_graph[global_idx_2_0][DAGNodeOperationType.READ] + del mock_graph[global_idx_2_0][DAGNodeOperationType.COMPUTE] + + mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_2_1][DAGNodeOperationType.READ] + ) + mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_1_1][DAGNodeOperationType.READ] + ) + mock_graph[global_idx_2_1][DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_2_1][DAGNodeOperationType.COMPUTE] + ) + mock_graph[global_idx_2_1][DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_2_1][DAGNodeOperationType.WRITE] + ) + mock_graph[global_idx_1_1][DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_1_1][DAGNodeOperationType.COMPUTE] + ) + mock_graph[global_idx_1_1][DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_1_1][DAGNodeOperationType.WRITE] + ) + mock_actor_to_candidates = { + fake_actor_1: [mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE]], + fake_actor_2: [mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE]], + } + + next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) + assert len(next_nodes) == 2 + assert next_nodes[0] in [ + mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE], + mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE], + ] + if next_nodes[0] == mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE]: + assert ( + next_nodes[1] == mock_graph[global_idx_2_1][DAGNodeOperationType.READ] + ) + elif next_nodes[0] == mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE]: + assert ( + next_nodes[1] == mock_graph[global_idx_1_1][DAGNodeOperationType.READ] + ) + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From 91897826bdb9e61b2f01e08160d59b6fe754f8e9 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 04:22:39 +0000 Subject: [PATCH 049/111] separate node generation and adding edge into two functions Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 95 +++++++++++++++++++++-------- 1 file changed, 69 insertions(+), 26 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 689fbb80176c2..28a905d9af19f 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1183,37 +1183,28 @@ def _get_or_compile( self._dag_submitter.start() self._dag_output_fetcher.start() - def _build_dag_node_operation_graph(self): + def _generate_dag_operation_graph_node( + self, + ) -> Dict["ray.actor.ActorHandle", List[List[DAGOperationGraphNode]]]: """ - Generate a DAG node operation graph based on the following rules: - - #1 Divide a DAG node into three DAGOperationGraphNodes: READ, COMPUTE, - and WRITE. Each DAGOperationGraphNode has a DAGNodeOperation. - #2 Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which - belong to the same task. - #3 Add an edge from COMPUTE with bind_index i to COMPUTE with bind_index - i+1 if they belong to the same actor. - #4 Add an edge from WRITE of the writer task to READ of the reader task. - - This is the step one of building an execution schedule for each actor. + Generate READ, COMPUTE, and WRITE operations for each DAG node. Returns: - A graph that each node is a DAGOperationGraphNode. + A dictionary that maps an actor handle to a list of lists of + DAGOperationGraphNode. For the same actor, the index of the + outer list corresponds to the index of the ExecutableTask in + the list of `executable_tasks` in `actor_to_executable_tasks`. + In the inner list, the order of operations is READ, COMPUTE, + and WRITE. """ assert self.idx_to_task assert self.actor_to_executable_tasks - assert not self.actor_to_execution_schedule - graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]] = {} - - from ray.dag import ( - ClassMethodNode, - MultiOutputNode, - ) + operation_nodes: Dict[ + "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] + ] = defaultdict(list) - # Step 1: Build a graph - for _, executable_tasks in self.actor_to_executable_tasks.items(): - prev_compute_node = None + for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): for local_idx, exec_task in enumerate(executable_tasks): # Divide a DAG node into three DAGOperationGraphNodes: READ, COMPUTE, # and WRITE. Each DAGOperationGraphNode has a DAGNodeOperation. @@ -1240,6 +1231,49 @@ def _build_dag_node_operation_graph(self): actor_handle, requires_nccl, ) + if operation_nodes[actor_handle] is None: + operation_nodes[actor_handle] = [] + operation_nodes[actor_handle].append( + [read_node, compute_node, write_node] + ) + return operation_nodes + + def _build_dag_node_operation_graph( + self, + actor_to_operation_nodes: Dict[ + "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] + ], + ): + """ + Generate a DAG node operation graph by adding edges based on the + following rules: + + #1 Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which + belong to the same task. + #2 Add an edge from COMPUTE with bind_index i to COMPUTE with bind_index + i+1 if they belong to the same actor. + #3 Add an edge from WRITE of the writer task to READ of the reader task. + + This is the step one of building an execution schedule for each actor. + + Returns: + A graph that each node is a DAGOperationGraphNode. + """ + assert self.idx_to_task + assert self.actor_to_executable_tasks + assert not self.actor_to_execution_schedule + + graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]] = {} + + for _, operation_nodes_list in actor_to_operation_nodes.items(): + for operation_nodes in operation_nodes_list: + prev_compute_node = None + idx = operation_nodes[0].idx + read_node, compute_node, write_node = ( + operation_nodes[0], + operation_nodes[1], + operation_nodes[2], + ) # Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which # belong to the same task. read_node.add_edge(compute_node) @@ -1249,12 +1283,18 @@ def _build_dag_node_operation_graph(self): if prev_compute_node is not None: prev_compute_node.add_edge(compute_node) prev_compute_node = compute_node + assert idx not in graph graph[idx] = { DAGNodeOperationType.READ: read_node, DAGNodeOperationType.COMPUTE: compute_node, DAGNodeOperationType.WRITE: write_node, } + from ray.dag import ( + ClassMethodNode, + MultiOutputNode, + ) + # Add an edge from WRITE of the writer task to READ of the reader task. for idx, task in self.idx_to_task.items(): if not isinstance(task.dag_node, ClassMethodNode): @@ -1273,8 +1313,9 @@ def _build_execution_schedule(self): Generate an execution schedule for each actor. The schedule is a list of DAGNodeOperation. - Step 1: Generate a DAG node operation graph. Refer to the function - `_build_dag_node_operation_graph` for more details. + Step 1: Generate a DAG node operation graph. Refer to the functions + `_generate_dag_operation_graph_node` and `_build_dag_node_operation_graph` + for more details. Step 2: Topological sort @@ -1291,7 +1332,9 @@ def _build_execution_schedule(self): See `test_execution_schedule` for more examples. """ - graph = self._build_dag_node_operation_graph() + # Step 1: Build a graph of DAGOperationGraphNode + actor_to_operation_nodes = self._generate_dag_operation_graph_node() + graph = self._build_dag_node_operation_graph(actor_to_operation_nodes) actor_to_candidates = defaultdict(list) for _, node_dict in graph.items(): From 56df8cc646cf7243f029f7f31a590d83ec486f81 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 04:29:54 +0000 Subject: [PATCH 050/111] add comments for _build_dag_node_operation_graph Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 28a905d9af19f..d05aafabd664c 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1200,7 +1200,7 @@ def _generate_dag_operation_graph_node( assert self.idx_to_task assert self.actor_to_executable_tasks - operation_nodes: Dict[ + actor_to_operation_nodes: Dict[ "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] ] = defaultdict(list) @@ -1231,12 +1231,10 @@ def _generate_dag_operation_graph_node( actor_handle, requires_nccl, ) - if operation_nodes[actor_handle] is None: - operation_nodes[actor_handle] = [] - operation_nodes[actor_handle].append( + actor_to_operation_nodes[actor_handle].append( [read_node, compute_node, write_node] ) - return operation_nodes + return actor_to_operation_nodes def _build_dag_node_operation_graph( self, @@ -1256,12 +1254,17 @@ def _build_dag_node_operation_graph( This is the step one of building an execution schedule for each actor. + Args: + actor_to_operation_nodes: A dictionary that maps an actor handle to + a list of lists of DAGOperationGraphNode. For the same actor, the + index of the outer list corresponds to the index of the ExecutableTask + in the list of `executable_tasks` in `actor_to_executable_tasks`. In + the inner list, the order of operations is READ, COMPUTE, and WRITE. + Returns: A graph that each node is a DAGOperationGraphNode. """ assert self.idx_to_task - assert self.actor_to_executable_tasks - assert not self.actor_to_execution_schedule graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]] = {} From 07125acb29981d9913b31f124f1dab24938f6ea1 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 17:57:16 +0000 Subject: [PATCH 051/111] add unit tests for _build_dag_node_operation_graph Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 2 +- .../experimental/test_execution_schedule.py | 177 +++++++++++++++++- 2 files changed, 176 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index d05aafabd664c..6038dc3b84ce1 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1269,8 +1269,8 @@ def _build_dag_node_operation_graph( graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]] = {} for _, operation_nodes_list in actor_to_operation_nodes.items(): + prev_compute_node = None for operation_nodes in operation_nodes_list: - prev_compute_node = None idx = operation_nodes[0].idx read_node, compute_node, write_node = ( operation_nodes[0], diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 5d666df882132..cd964c49bd9d8 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -8,13 +8,13 @@ import ray.cluster_utils from ray.experimental.channel.torch_tensor_type import TorchTensorType from ray.tests.conftest import * # noqa -from ray.dag import InputNode, MultiOutputNode +from ray.dag import InputNode, MultiOutputNode, ClassMethodNode from ray.dag.dag_node_operation import ( DAGNodeOperationType, DAGOperationGraphNode, DAGNodeOperation, ) -from ray.dag.compiled_dag_node import _select_next_nodes +from ray.dag.compiled_dag_node import _select_next_nodes, CompiledDAG, CompiledTask import torch from typing import List from dataclasses import dataclass, field @@ -668,6 +668,179 @@ def test_two_nccl_writes(self): ) +def mock_init(self): + pass + + +class TestBuildDAGNodeOperationGraph: + """ + Test whether `_build_dag_node_operation_graph` function adds the correct + edges between the nodes in the operation graph. + """ + + def check_edges_between_read_compute_write( + self, graph, global_idx, expected_num_edges + ): + read_node = graph[global_idx][DAGNodeOperationType.READ] + compute_node = graph[global_idx][DAGNodeOperationType.COMPUTE] + write_node = graph[global_idx][DAGNodeOperationType.WRITE] + + for idx, node in enumerate([read_node, compute_node, write_node]): + assert node.in_degree == expected_num_edges[idx][0] + assert len(node.out_edges) == expected_num_edges[idx][1] + + assert (global_idx, DAGNodeOperationType.COMPUTE) in read_node.out_edges + assert (global_idx, DAGNodeOperationType.READ) in compute_node.in_edges + assert (global_idx, DAGNodeOperationType.WRITE) in compute_node.out_edges + assert (global_idx, DAGNodeOperationType.COMPUTE) in write_node.in_edges + + def check_edge_between_writer_and_reader(self, graph, writer_idx, reader_idx): + write_node = graph[writer_idx][DAGNodeOperationType.WRITE] + read_node = graph[reader_idx][DAGNodeOperationType.READ] + + assert (reader_idx, DAGNodeOperationType.READ) in write_node.out_edges + assert (writer_idx, DAGNodeOperationType.WRITE) in read_node.in_edges + + def check_edge_between_compute_nodes(self, graph, global_idx_1, global_idx_2): + compute_node_1 = graph[global_idx_1][DAGNodeOperationType.COMPUTE] + compute_node_2 = graph[global_idx_2][DAGNodeOperationType.COMPUTE] + + assert (global_idx_2, DAGNodeOperationType.COMPUTE) in compute_node_1.out_edges + assert (global_idx_1, DAGNodeOperationType.COMPUTE) in compute_node_2.in_edges + + def test_edges_between_read_compute_write(self, monkeypatch): + """ + driver -> fake_actor.op -> driver + + This test case aims to verify whether the function correctly adds edges + between READ/COMPUTE and COMPUTE/WRITE operations on the same actor. + """ + monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) + monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) + + compiled_dag = CompiledDAG() + compiled_dag.idx_to_task = { + 0: CompiledTask(0, InputNode()), + 1: CompiledTask(1, ClassMethodNode()), + 2: CompiledTask(2, MultiOutputNode()), + } + + fake_actor = "fake_actor" + global_idx = 1 + actor_to_operation_nodes = { + fake_actor: [ + list( + generate_dag_graph_nodes(0, global_idx, fake_actor, False).values() + ) + ] + } + graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + assert len(graph) == 1 + assert len(graph[global_idx]) == 3 + + self.check_edges_between_read_compute_write( + graph, global_idx, [(0, 1), (1, 1), (1, 0)] + ) + + def test_edge_between_writer_and_reader(self, monkeypatch): + """ + driver -> fake_actor_1.op -> fake_actor_2.op -> driver + + This test case aims to verify whether the function correctly adds an edge + from the writer's WRITE operation to the reader's READ operation. + """ + monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) + monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) + + fake_actor_1, global_idx_1 = "fake_actor_1", 1 + fake_actor_2, global_idx_2 = "fake_actor_2", 2 + compiled_dag = CompiledDAG() + compiled_dag.idx_to_task = { + 0: CompiledTask(0, InputNode()), + 1: CompiledTask(1, ClassMethodNode()), + 2: CompiledTask(2, ClassMethodNode()), + 3: CompiledTask(3, MultiOutputNode()), + } + compiled_dag.idx_to_task[1].downstream_node_idxs = {2: fake_actor_2} + + actor_to_operation_nodes = { + fake_actor_1: [ + list( + generate_dag_graph_nodes( + 0, global_idx_1, fake_actor_1, False + ).values() + ) + ], + fake_actor_2: [ + list( + generate_dag_graph_nodes( + 0, global_idx_2, fake_actor_2, False + ).values() + ) + ], + } + graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + assert len(graph) == 2 + assert len(graph[global_idx_1]) == 3 + assert len(graph[global_idx_2]) == 3 + + self.check_edges_between_read_compute_write( + graph, global_idx_1, [(0, 1), (1, 1), (1, 1)] + ) + self.check_edges_between_read_compute_write( + graph, global_idx_2, [(1, 1), (1, 1), (1, 0)] + ) + self.check_edge_between_writer_and_reader(graph, global_idx_1, global_idx_2) + + def test_edge_between_compute_nodes(self, monkeypatch): + """ + driver -> fake_actor.op -> fake_actor.op -> driver + """ + monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) + monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) + + fake_actor = "fake_actor" + global_idx_1, global_idx_2 = 1, 2 + compiled_dag = CompiledDAG() + compiled_dag.idx_to_task = { + 0: CompiledTask(0, InputNode()), + global_idx_1: CompiledTask(global_idx_1, ClassMethodNode()), + global_idx_2: CompiledTask(global_idx_2, ClassMethodNode()), + 3: CompiledTask(3, MultiOutputNode()), + } + compiled_dag.idx_to_task[global_idx_1].downstream_node_idxs = { + global_idx_2: fake_actor + } + + actor_to_operation_nodes = { + fake_actor: [ + list( + generate_dag_graph_nodes( + 0, global_idx_1, fake_actor, False + ).values() + ), + list( + generate_dag_graph_nodes( + 1, global_idx_2, fake_actor, False + ).values() + ), + ], + } + graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + assert len(graph) == 2 + assert len(graph[global_idx_1]) == 3 + assert len(graph[global_idx_2]) == 3 + + self.check_edges_between_read_compute_write( + graph, global_idx_1, [(0, 1), (1, 2), (1, 1)] + ) + self.check_edges_between_read_compute_write( + graph, global_idx_2, [(1, 1), (2, 1), (1, 0)] + ) + self.check_edge_between_writer_and_reader(graph, global_idx_1, global_idx_2) + self.check_edge_between_compute_nodes(graph, global_idx_1, global_idx_2) + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From 5c3750ed58716da39fb2b93e17b9af225c9a0fee Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 20:05:25 +0000 Subject: [PATCH 052/111] address comments Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 18 ++++++++++-------- python/ray/dag/dag_node_operation.py | 4 ++++ 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 6038dc3b84ce1..ee029910a06bb 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -93,10 +93,10 @@ def do_exec_tasks( tasks: List["ExecutableTask"], schedule: List[DAGNodeOperation], ) -> None: - """Generic actor method to begin executing the tasks belonging to an actor. - This runs an infinite loop to run each task in turn (following the order specified - in the list): reading input channel(s), executing the given taks, and writing output - channel(s). It only exits if the actor dies or an exception is thrown. + """A generic actor method to begin executing the operations belonging to an + actor. This runs an infinite loop to execute each DAGNodeOperation in the + order specified by the schedule. It exits only if the actor dies or an + exception is thrown. Args: tasks: the executable tasks corresponding to the actor methods. @@ -1241,7 +1241,7 @@ def _build_dag_node_operation_graph( actor_to_operation_nodes: Dict[ "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] ], - ): + ) -> Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]]: """ Generate a DAG node operation graph by adding edges based on the following rules: @@ -1262,7 +1262,10 @@ def _build_dag_node_operation_graph( the inner list, the order of operations is READ, COMPUTE, and WRITE. Returns: - A graph that each node is a DAGOperationGraphNode. + A graph where each node is a DAGOperationGraphNode. The key is the index + of the task in idx_to_task, and the value is a dictionary that maps the + DAGNodeOperationType (READ, COMPUTE, or WRITE) to the corresponding + DAGOperationGraphNode. """ assert self.idx_to_task @@ -1327,8 +1330,7 @@ def _build_execution_schedule(self): Then, put the selected nodes into the corresponding actors' schedules. - The goal of the above rules is to build a schedule with fewer bubbles. The - schedule should be intuitive to users, meaning that the execution should + The schedule should be intuitive to users, meaning that the execution should perform operations in ascending order of `bind_index` as much as possible. [Example]: diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index b55696cc92128..7ede4acabb289 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -93,5 +93,9 @@ def __hash__(self): return hash((self.operation, self.idx)) def add_edge(self, out_node: "DAGOperationGraphNode"): + """ + Add an edge from this node to `out_node`. An edge is a tuple of + the operation's index and type. + """ self.out_edges.add((out_node.idx, out_node.operation.type)) out_node.in_edges.add((self.idx, self.operation.type)) From 89341822ae87d9cb61acacbaa61422a70cbadd09 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 20:26:03 +0000 Subject: [PATCH 053/111] add comments Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 7 +- .../experimental/test_execution_schedule.py | 78 +++++++++++++++---- 2 files changed, 68 insertions(+), 17 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 7ede4acabb289..23f85e297291e 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -85,9 +85,10 @@ def __eq__(self, other): only when they have the same `idx` and `type`. """ assert self.actor_handle == other.actor_handle - if self.operation.idx == other.operation.idx: - return self.operation.type == other.operation.type - return False + return ( + self.operation.idx == other.operation.idx + and self.operation.type == other.operation.type + ) def __hash__(self): return hash((self.operation, self.idx)) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index cd964c49bd9d8..ad0c7f488761e 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -16,7 +16,7 @@ ) from ray.dag.compiled_dag_node import _select_next_nodes, CompiledDAG, CompiledTask import torch -from typing import List +from typing import List, Dict, Tuple from dataclasses import dataclass, field from collections import deque, defaultdict from ray.actor import ActorHandle @@ -675,12 +675,30 @@ def mock_init(self): class TestBuildDAGNodeOperationGraph: """ Test whether `_build_dag_node_operation_graph` function adds the correct - edges between the nodes in the operation graph. + edges between the nodes in the operation graph base on the 3 rules mentioned + in the doc string of `_build_dag_node_operation_graph`. """ def check_edges_between_read_compute_write( - self, graph, global_idx, expected_num_edges + self, + graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], + global_idx: int, + expected_num_edges: List[Tuple[int, int]], ): + """ + Check whether edges from READ to COMPUTE, and from COMPUTE to WRITE, + belonging to the same task are added. + + Args: + graph: The operation graph generated by `_build_dag_node_operation_graph`. + global_idx: The global index of the task used to access the task in + `idx_to_task`. + expected_num_edges: A list of tuples where each tuple contains the expected + number of in-edges and out-edges for READ, COMPUTE, and WRITE + operations. + """ + assert len(expected_num_edges) == 3 + assert len(graph[global_idx]) == 3 read_node = graph[global_idx][DAGNodeOperationType.READ] compute_node = graph[global_idx][DAGNodeOperationType.COMPUTE] write_node = graph[global_idx][DAGNodeOperationType.WRITE] @@ -694,14 +712,47 @@ def check_edges_between_read_compute_write( assert (global_idx, DAGNodeOperationType.WRITE) in compute_node.out_edges assert (global_idx, DAGNodeOperationType.COMPUTE) in write_node.in_edges - def check_edge_between_writer_and_reader(self, graph, writer_idx, reader_idx): - write_node = graph[writer_idx][DAGNodeOperationType.WRITE] - read_node = graph[reader_idx][DAGNodeOperationType.READ] + def check_edge_between_writer_and_reader( + self, + graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], + writer_global_idx: int, + reader_global_idx: int, + ): + """ + Check whether the edge from writer's WRITE to reader's READ operation is added. + + Args: + graph: The operation graph generated by `_build_dag_node_operation_graph`. + writer_global_idx: The global index of the task used to access the task + that the writer belongs to in `idx_to_task`. + reader_global_idx: The global index of the task used to access the task + that the reader belongs to in `idx_to_task`. + """ + write_node = graph[writer_global_idx][DAGNodeOperationType.WRITE] + read_node = graph[reader_global_idx][DAGNodeOperationType.READ] - assert (reader_idx, DAGNodeOperationType.READ) in write_node.out_edges - assert (writer_idx, DAGNodeOperationType.WRITE) in read_node.in_edges + assert (reader_global_idx, DAGNodeOperationType.READ) in write_node.out_edges + assert (writer_global_idx, DAGNodeOperationType.WRITE) in read_node.in_edges - def check_edge_between_compute_nodes(self, graph, global_idx_1, global_idx_2): + def check_edge_between_compute_nodes( + self, + graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], + global_idx_1: int, + global_idx_2: int, + ): + """ + Check whether the edge from COMPUTE with `bind_index` i to COMPUTE with + `bind_index` i+1 if they belong to the same actor. + + Args: + graph: The operation graph generated by `_build_dag_node_operation_graph`. + global_idx_1: The global index of the task used to access the task in + `idx_to_task`. + global_idx_2: The global index of the task used to access the task in + `idx_to_task`. Note that both tasks belong to the same actor, and the + `bind_index` of the second task is equal to the `bind_index` of the + first task plus one. + """ compute_node_1 = graph[global_idx_1][DAGNodeOperationType.COMPUTE] compute_node_2 = graph[global_idx_2][DAGNodeOperationType.COMPUTE] @@ -736,7 +787,6 @@ def test_edges_between_read_compute_write(self, monkeypatch): } graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) assert len(graph) == 1 - assert len(graph[global_idx]) == 3 self.check_edges_between_read_compute_write( graph, global_idx, [(0, 1), (1, 1), (1, 0)] @@ -781,8 +831,6 @@ def test_edge_between_writer_and_reader(self, monkeypatch): } graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) assert len(graph) == 2 - assert len(graph[global_idx_1]) == 3 - assert len(graph[global_idx_2]) == 3 self.check_edges_between_read_compute_write( graph, global_idx_1, [(0, 1), (1, 1), (1, 1)] @@ -795,6 +843,10 @@ def test_edge_between_writer_and_reader(self, monkeypatch): def test_edge_between_compute_nodes(self, monkeypatch): """ driver -> fake_actor.op -> fake_actor.op -> driver + + This test case aims to verify whether the function correctly adds an edge + from the COMPUTE operation with `bind_index` i to the COMPUTE operation with + `bind_index` i+1 if they belong to the same actor. """ monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) @@ -828,8 +880,6 @@ def test_edge_between_compute_nodes(self, monkeypatch): } graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) assert len(graph) == 2 - assert len(graph[global_idx_1]) == 3 - assert len(graph[global_idx_2]) == 3 self.check_edges_between_read_compute_write( graph, global_idx_1, [(0, 1), (1, 2), (1, 1)] From 09fcc2d4dff646f0338ff627237cab2592cfc0e2 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 20:43:00 +0000 Subject: [PATCH 054/111] add unit test Signed-off-by: Kai-Hsun Chen --- .../experimental/test_execution_schedule.py | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index ad0c7f488761e..335fd883b7d1b 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -890,6 +890,82 @@ def test_edge_between_compute_nodes(self, monkeypatch): self.check_edge_between_writer_and_reader(graph, global_idx_1, global_idx_2) self.check_edge_between_compute_nodes(graph, global_idx_1, global_idx_2) + def test_two_actors(self, monkeypatch): + """ + driver -> fake_actor_1.op -> fake_actor_2.op -> driver + | | + -> fake_actor_2.op -> fake_actor_1.op - + + This test includes two actors, each with two tasks. The + test case covers all three rules for adding edges between + operation nodes in the operation graph. + """ + monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) + monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) + + fake_actor_1, global_idx_1, global_idx_3 = "fake_actor_1", 1, 3 + fake_actor_2, global_idx_2, global_idx_4 = "fake_actor_2", 2, 4 + + compiled_dag = CompiledDAG() + compiled_dag.idx_to_task = { + 0: CompiledTask(0, InputNode()), + global_idx_1: CompiledTask(global_idx_1, ClassMethodNode()), + global_idx_2: CompiledTask(global_idx_2, ClassMethodNode()), + global_idx_3: CompiledTask(global_idx_3, ClassMethodNode()), + global_idx_4: CompiledTask(global_idx_4, ClassMethodNode()), + 5: CompiledTask(5, MultiOutputNode()), + } + compiled_dag.idx_to_task[global_idx_1].downstream_node_idxs = { + global_idx_4: fake_actor_2 + } + compiled_dag.idx_to_task[global_idx_2].downstream_node_idxs = { + global_idx_3: fake_actor_1 + } + + actor_to_operation_nodes = { + fake_actor_1: [ + list( + generate_dag_graph_nodes( + 0, global_idx_1, fake_actor_1, False + ).values() + ), + list( + generate_dag_graph_nodes( + 1, global_idx_3, fake_actor_1, False + ).values() + ), + ], + fake_actor_2: [ + list( + generate_dag_graph_nodes( + 0, global_idx_2, fake_actor_2, False + ).values() + ), + list( + generate_dag_graph_nodes( + 1, global_idx_4, fake_actor_2, False + ).values() + ), + ], + } + graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + assert len(graph) == 4 + + self.check_edges_between_read_compute_write( + graph, global_idx_1, [(0, 1), (1, 2), (1, 1)] + ) + self.check_edges_between_read_compute_write( + graph, global_idx_2, [(0, 1), (1, 2), (1, 1)] + ) + self.check_edges_between_read_compute_write( + graph, global_idx_3, [(1, 1), (2, 1), (1, 0)] + ) + self.check_edges_between_read_compute_write( + graph, global_idx_4, [(1, 1), (2, 1), (1, 0)] + ) + self.check_edge_between_writer_and_reader(graph, global_idx_1, global_idx_4) + self.check_edge_between_writer_and_reader(graph, global_idx_2, global_idx_3) + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): From 98df0695d16655dd656850627636e10c131e15e3 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 21:15:08 +0000 Subject: [PATCH 055/111] move tests to GPU instance Signed-off-by: Kai-Hsun Chen --- python/ray/dag/BUILD | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 7c7abfef21b56..1298d3ca1196b 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -103,7 +103,6 @@ py_test_module_list( size = "medium", files = [ "tests/experimental/test_detect_deadlock_dag.py", - "tests/experimental/test_execution_schedule.py", "tests/experimental/test_multi_node_dag.py", "tests/experimental/test_torch_tensor_dag.py", ], @@ -147,3 +146,21 @@ py_test( ], deps = ["//:ray_lib"], ) + +py_test( + name = "test_execution_schedule", + size = "medium", + srcs = [ + "tests/experimental/test_execution_schedule.py", + ], + env = {"RAY_PYTEST_USE_GPU": "1"}, + main = "tests/experimental/test_execution_schedule.py", + tags = [ + "accelerated_dag", + "exclusive", + "multi_gpu", + "no_windows", + "team:core", + ], + deps = ["//:ray_lib"], +) From 7b06b414b8d24e29bff7bd929cbac75113be2a32 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 21:21:09 +0000 Subject: [PATCH 056/111] use existing GPU instance Signed-off-by: Kai-Hsun Chen --- python/ray/dag/BUILD | 22 ++-------------------- 1 file changed, 2 insertions(+), 20 deletions(-) diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 1298d3ca1196b..3cbd047883042 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -130,31 +130,13 @@ py_test_module_list( ) py_test( - name = "test_torch_tensor_dag_gpu", + name = "Accelerated DAG GPU tests", size = "medium", - srcs = [ + files = [ "tests/experimental/test_torch_tensor_dag.py", - ], - env = {"RAY_PYTEST_USE_GPU": "1"}, - main = "tests/experimental/test_torch_tensor_dag.py", - tags = [ - "accelerated_dag", - "exclusive", - "multi_gpu", - "no_windows", - "team:core", - ], - deps = ["//:ray_lib"], -) - -py_test( - name = "test_execution_schedule", - size = "medium", - srcs = [ "tests/experimental/test_execution_schedule.py", ], env = {"RAY_PYTEST_USE_GPU": "1"}, - main = "tests/experimental/test_execution_schedule.py", tags = [ "accelerated_dag", "exclusive", From e664e96dad6cf2469cc211340602c415c2157c93 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 22:08:35 +0000 Subject: [PATCH 057/111] fix BUILD Signed-off-by: Kai-Hsun Chen --- python/ray/dag/BUILD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 3cbd047883042..3d635d2ce08b8 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -129,7 +129,7 @@ py_test_module_list( deps = ["//:ray_lib"], ) -py_test( +py_test_module_list( name = "Accelerated DAG GPU tests", size = "medium", files = [ From 09595996a08439b4bcab27f062bd422f50e72817 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 22:42:36 +0000 Subject: [PATCH 058/111] fix BUILD Signed-off-by: Kai-Hsun Chen --- python/ray/dag/BUILD | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 3d635d2ce08b8..1298d3ca1196b 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -129,14 +129,32 @@ py_test_module_list( deps = ["//:ray_lib"], ) -py_test_module_list( - name = "Accelerated DAG GPU tests", +py_test( + name = "test_torch_tensor_dag_gpu", size = "medium", - files = [ + srcs = [ "tests/experimental/test_torch_tensor_dag.py", + ], + env = {"RAY_PYTEST_USE_GPU": "1"}, + main = "tests/experimental/test_torch_tensor_dag.py", + tags = [ + "accelerated_dag", + "exclusive", + "multi_gpu", + "no_windows", + "team:core", + ], + deps = ["//:ray_lib"], +) + +py_test( + name = "test_execution_schedule", + size = "medium", + srcs = [ "tests/experimental/test_execution_schedule.py", ], env = {"RAY_PYTEST_USE_GPU": "1"}, + main = "tests/experimental/test_execution_schedule.py", tags = [ "accelerated_dag", "exclusive", From 2199bdae04c04bdfed024b2d859ee6ca4265eec1 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 7 Aug 2024 22:59:15 +0000 Subject: [PATCH 059/111] add comments Signed-off-by: Kai-Hsun Chen --- .../experimental/test_execution_schedule.py | 52 ++++++++++++++++--- 1 file changed, 44 insertions(+), 8 deletions(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 335fd883b7d1b..969e88024c5d5 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -29,12 +29,24 @@ @dataclass class PipelineConfig: + """ + pp_size: Number of pipeline parallel workers. + num_micro_batches: Number of micro-batches. + """ + pp_size: int num_micro_batches: int @dataclass class PipelineUnit: + """ + op: Operation type (FWD or BWD). + pp_rank: Pipeline parallel rank. + batch_id: Batch ID. + uid: Unique ID for the pipeline unit. + """ + op: str pp_rank: int batch_id: int @@ -47,7 +59,14 @@ def __repr__(self) -> str: return self.uid -def generate_1f1b_schedule(config) -> List[List[PipelineUnit]]: +def generate_1f1b_schedule(config: PipelineConfig) -> List[List[PipelineUnit]]: + """ + Args: + config: Pipeline configuration. + Returns: + schedule: List of pipeline units for 1F1B pipeline parallelism. Each + inner list represents the schedule for a pipeline parallel worker. + """ pp_size = config.pp_size num_micro_batches = config.num_micro_batches @@ -83,15 +102,27 @@ def __init__( config: PipelineConfig, schedule: List[List[PipelineUnit]], blocks: List[ActorHandle], - compile_dag: bool = True, ) -> None: + """ + Args: + config: Pipeline configuration. + schedule: List of pipeline units. Each inner list represents the + schedule for a pipeline parallel worker. + blocks: List of actors representing pipeline parallel workers. + """ self.config = config self.blocks = blocks self.generate_pipeline_schedules(schedule) - self.compile_dag = compile_dag self.dag = self.build_dag() - def generate_pipeline_schedules(self, schedule): + def generate_pipeline_schedules(self, schedule: List[List[PipelineUnit]]): + """ + Convert per-worker schedule to per-batch schedule. + + Args: + schedule: List of pipeline units. Each inner list represents the + schedule for a pipeline parallel worker. + """ self.id_to_unit = dict() self.stage_schedules = defaultdict(list) self.batch_schedules = defaultdict(list) @@ -115,6 +146,10 @@ def generate_pipeline_schedules(self, schedule): self.batch_schedules[batch_id] = fwd_units + bwd_units def build_dependency_graph(self): + """ + Add dependencies between pipeline units based on: + (1) Per-batch schedule and (2) Per-worker schedule. + """ graph = defaultdict(set) reversed_graph = defaultdict(set) @@ -129,6 +164,9 @@ def build_dependency_graph(self): return graph, reversed_graph def build_dag(self): + """ + Build accelerated DAG for the pipeline model. + """ graph, reversed_graph = self.build_dependency_graph() dag_nodes = dict() # Cache DAG Node for each unit @@ -182,10 +220,8 @@ def build_dag(self): queue.append(target_uid) dag = MultiOutputNode(output_nodes) - - if self.compile_dag: - dag = dag.experimental_compile() - return dag + compiled_dag = dag.experimental_compile() + return compiled_dag def step(self, input_batches): return ray.get(self.dag.execute(input_batches)) From 383b877cb353c958e12e08ea0af2c4d0f23b0519 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 02:23:30 +0000 Subject: [PATCH 060/111] add type hints Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 10 +++++----- python/ray/dag/dag_node_operation.py | 6 +++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index ee029910a06bb..c996ba35f2273 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -403,15 +403,15 @@ def prepare(self): self.input_reader.start() self.output_writer.start() - def set_cache(self, data): + def set_cache(self, data: Any): self._cache = data - def reset_cache(self): + def reset_cache(self) -> Any: data = self._cache self._cache = None return data - def _read(self): + def _read(self) -> bool: """ Read input data from upstream DAG nodes and cache the intermediate result. """ @@ -423,7 +423,7 @@ def _read(self): # Channel closed. Exit the loop. return True - def _compute(self, class_handle): + def _compute(self, class_handle) -> bool: """ Retrieve the intermediate result from the READ operation and perform the computation. Then, cache the new intermediate result. The caller must ensure @@ -453,7 +453,7 @@ def _compute(self, class_handle): self.set_cache(output_val) return False - def _write(self): + def _write(self) -> bool: """ Retrieve the intermediate result from the COMPUTE operation and write to its downstream DAG nodes. The caller must ensure that the last operation executed diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 23f85e297291e..50a2ff2901793 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -1,10 +1,10 @@ from functools import total_ordering from enum import Enum +from typing import Set, Tuple from ray.util.annotations import DeveloperAPI import ray -@DeveloperAPI class DAGNodeOperationType(Enum): """ There are three types of operations that a DAG node can perform: @@ -62,8 +62,8 @@ def __init__( self.idx = idx self.actor_handle = actor_handle self.requires_nccl = requires_nccl - self.in_edges = set() - self.out_edges = set() + self.in_edges: Set[Tuple[int, DAGNodeOperationType]] = set() + self.out_edges: Set[Tuple[int, DAGNodeOperationType]] = set() @property def in_degree(self) -> int: From 50ff5c1385be55a1220fd8b9ef05f28b89444c03 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 03:38:19 +0000 Subject: [PATCH 061/111] use actor_id instead of actor handle Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 31 ++++++++++++------- .../experimental/test_execution_schedule.py | 27 ++++++++++------ 2 files changed, 36 insertions(+), 22 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index c996ba35f2273..6b6325012fbe9 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -139,7 +139,7 @@ def _wrap_exception(exc): def _select_next_nodes( - actor_to_candidates: Dict["ray.actor.ActorHandle", List[DAGOperationGraphNode]], + actor_to_candidates: Dict["ray._raylet.ActorID", List[DAGOperationGraphNode]], graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], ): """ @@ -153,10 +153,10 @@ def _select_next_nodes( acceptable. For the implementation details, we maintain a priority queue for each actor, where the head of the priority queue is the node with the smallest `bind_index`. - #2 If #1 cannot be satisfied, it means that all candidate nodes are - NCCL write nodes. In this case, select the one that is the head of the - priority queue and its downstream nodes, regardless of whether the - downstream nodes are heads of their priority queues or not. + #2 If #1 cannot be satisfied, it means that all candidate nodes are NCCL write + nodes. In this case, select the one at the head of the priority queue and + its downstream nodes, which are NCCL read nodes, regardless of whether the + downstream nodes are heads of their own priority queues. This function may return multiple nodes if they are NCCL nodes. In that case, this function only removes the NCCL write node, which is also the head of a @@ -165,7 +165,7 @@ def _select_next_nodes( more than once. Args: - actor_to_candidates: A dictionary mapping an actor handle to a list of + actor_to_candidates: A dictionary mapping an actor id to a list of candidate nodes with zero in-degree. graph: A dictionary mapping the index of a task to a dictionary of its DAGOperationGraphNodes for different operations. @@ -183,7 +183,9 @@ def _select_next_nodes( first_nccl_node = candidates[0] assert first_nccl_node is not None - next_nodes.append(heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle])) + next_nodes.append( + heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) + ) for downstream_node_metadata in first_nccl_node.out_edges: downstream_node = graph[downstream_node_metadata[0]][ downstream_node_metadata[1] @@ -1341,11 +1343,15 @@ def _build_execution_schedule(self): actor_to_operation_nodes = self._generate_dag_operation_graph_node() graph = self._build_dag_node_operation_graph(actor_to_operation_nodes) - actor_to_candidates = defaultdict(list) + actor_to_candidates: Dict[ + "ray._raylet.ActorID", List[DAGOperationGraphNode] + ] = defaultdict(list) for _, node_dict in graph.items(): for _, node in node_dict.items(): if node.in_degree == 0: - heapq.heappush(actor_to_candidates[node.actor_handle], node) + heapq.heappush( + actor_to_candidates[node.actor_handle._actor_id], node + ) visited_nodes = set() @@ -1364,13 +1370,14 @@ def _build_execution_schedule(self): out_node.in_edges.remove((node.idx, node.operation.type)) if out_node.in_degree == 0: heapq.heappush( - actor_to_candidates[out_node.actor_handle], out_node + actor_to_candidates[out_node.actor_handle._actor_id], + out_node, ) delete_keys = [] - for actor_handle, candidates in actor_to_candidates.items(): + for actor_id, candidates in actor_to_candidates.items(): if len(candidates) == 0: - delete_keys.append(actor_handle) + delete_keys.append(actor_id) for key in delete_keys: del actor_to_candidates[key] diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 969e88024c5d5..2d745df489cf3 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -251,6 +251,14 @@ def no_op_two(self, value1, value2): return value1, value2 +def mock_actor_handle_init(self, actor_id: str): + self._ray_actor_id = actor_id + + +def mock_init(self): + pass + + @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): """ @@ -583,7 +591,7 @@ def test_two_candidates_on_same_actor(self): assert len(next_nodes) == 1 assert next_nodes[0] == dag_node_1 - def test_only_one_nccl_write(self): + def test_only_one_nccl_write(self, monkeypatch): """ Simulate the case where there is only one candidate which is a NCCL WRITE operation. In this case, `_select_next_nodes` should return both @@ -596,8 +604,9 @@ def test_only_one_nccl_write(self): READ and COMPUTE operations on fake_actor_1 have already been added to the execution schedule. """ - fake_actor_1, global_idx_1, local_idx_1 = "fake_actor_1", 1, 0 - fake_actor_2, global_idx_2, local_idx_2 = "fake_actor_2", 2, 0 + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + fake_actor_1, global_idx_1, local_idx_1 = ActorHandle("fake_actor_1"), 1, 0 + fake_actor_2, global_idx_2, local_idx_2 = ActorHandle("fake_actor_2"), 2, 0 mock_graph = { global_idx_1: generate_dag_graph_nodes( local_idx_1, global_idx_1, fake_actor_1, True @@ -625,7 +634,7 @@ def test_only_one_nccl_write(self): assert next_nodes[0] == mock_graph[global_idx_1][DAGNodeOperationType.WRITE] assert next_nodes[1] == mock_graph[global_idx_2][DAGNodeOperationType.READ] - def test_two_nccl_writes(self): + def test_two_nccl_writes(self, monkeypatch): """ Simulate a scenario where there are two candidates that are NCCL WRITE operations. In this case, _select_next_nodes can choose either of the @@ -640,10 +649,12 @@ def test_two_nccl_writes(self): and COMPUTE operations on both the DAG nodes with smaller bind_index on fake_actor_1 and fake_actor_2 have already been added to the execution schedule. """ - fake_actor_1 = "fake_actor_1" + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + + fake_actor_1 = ActorHandle("fake_actor_1") global_idx_1_0, local_idx_1_0 = 1, 0 global_idx_1_1, local_idx_1_1 = 3, 1 - fake_actor_2 = "fake_actor_2" + fake_actor_2 = ActorHandle("fake_actor_2") global_idx_2_0, local_idx_2_0 = 2, 0 global_idx_2_1, local_idx_2_1 = 4, 1 mock_graph = { @@ -704,10 +715,6 @@ def test_two_nccl_writes(self): ) -def mock_init(self): - pass - - class TestBuildDAGNodeOperationGraph: """ Test whether `_build_dag_node_operation_graph` function adds the correct From 8667ed48d4c5724e9da69035864e05c501c61d56 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 04:01:33 +0000 Subject: [PATCH 062/111] rename cache to intermediate_buffer Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 22 +++++++++++----------- python/ray/dag/dag_node_operation.py | 1 + 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 6b6325012fbe9..b863a6506c892 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -388,7 +388,7 @@ def __init__( # Store the intermediate result of a READ or COMPUTE operation. # The result of a READ operation will be used by a COMPUTE operation, # and the result of a COMPUTE operation will be used by a WRITE operation. - self._cache = None + self._intermediate_buffer = None def cancel(self): self.input_reader.close() @@ -405,12 +405,12 @@ def prepare(self): self.input_reader.start() self.output_writer.start() - def set_cache(self, data: Any): - self._cache = data + def set_intermediate_buffer(self, data: Any): + self._intermediate_buffer = data - def reset_cache(self) -> Any: - data = self._cache - self._cache = None + def reset_intermediate_buffer(self) -> Any: + data = self._intermediate_buffer + self._intermediate_buffer = None return data def _read(self) -> bool: @@ -419,7 +419,7 @@ def _read(self) -> bool: """ try: res = self.input_reader.read() - self.set_cache(res) + self.set_intermediate_buffer(res) return False except RayChannelError: # Channel closed. Exit the loop. @@ -432,7 +432,7 @@ def _compute(self, class_handle) -> bool: that the last operation executed is READ so that the function retrieves the correct intermediate result. """ - res = self.reset_cache() + res = self.reset_intermediate_buffer() method = getattr(class_handle, self.method_name) try: _process_return_vals(res, return_single_output=False) @@ -441,7 +441,7 @@ def _compute(self, class_handle) -> bool: # Propagate it and skip the actual task. We don't need to wrap the # exception in a RayTaskError here because it has already been wrapped # by the previous task. - self.set_cache(exc) + self.set_intermediate_buffer(exc) return False resolved_inputs = [] @@ -452,7 +452,7 @@ def _compute(self, class_handle) -> bool: output_val = method(*resolved_inputs, **self.resolved_kwargs) except Exception as exc: output_val = _wrap_exception(exc) - self.set_cache(output_val) + self.set_intermediate_buffer(output_val) return False def _write(self) -> bool: @@ -461,7 +461,7 @@ def _write(self) -> bool: downstream DAG nodes. The caller must ensure that the last operation executed is COMPUTE so that the function retrieves the correct intermediate result. """ - output_val = self.reset_cache() + output_val = self.reset_intermediate_buffer() try: self.output_writer.write(output_val) return False diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 50a2ff2901793..6f47033b0cb40 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -5,6 +5,7 @@ import ray +@DeveloperAPI class DAGNodeOperationType(Enum): """ There are three types of operations that a DAG node can perform: From 980f5a137fe3674c7ea72e2c180e1297e5256457 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 04:04:41 +0000 Subject: [PATCH 063/111] add asserts Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index b863a6506c892..b5e2823b35d5c 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -178,6 +178,7 @@ def _select_next_nodes( or candidates[0].operation.type != DAGNodeOperationType.WRITE ): next_nodes.append(heapq.heappop(candidates)) + assert len(next_nodes) == 1 return next_nodes if first_nccl_node is None: first_nccl_node = candidates[0] @@ -191,6 +192,7 @@ def _select_next_nodes( downstream_node_metadata[1] ] next_nodes.append(downstream_node) + assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) return next_nodes @@ -388,7 +390,7 @@ def __init__( # Store the intermediate result of a READ or COMPUTE operation. # The result of a READ operation will be used by a COMPUTE operation, # and the result of a COMPUTE operation will be used by a WRITE operation. - self._intermediate_buffer = None + self._intermediate_buffer: Any = None def cancel(self): self.input_reader.close() From cafd691ee8215d7e86692423156b484a3bcd536d Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 04:08:42 +0000 Subject: [PATCH 064/111] add asserts Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index b5e2823b35d5c..624831999ab31 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -170,8 +170,8 @@ def _select_next_nodes( graph: A dictionary mapping the index of a task to a dictionary of its DAGOperationGraphNodes for different operations. """ - next_nodes = [] - first_nccl_node = None + next_nodes: List[DAGOperationGraphNode] = [] + first_nccl_node: Optional[DAGOperationGraphNode] = None for _, candidates in actor_to_candidates.items(): if ( not candidates[0].requires_nccl @@ -181,6 +181,7 @@ def _select_next_nodes( assert len(next_nodes) == 1 return next_nodes if first_nccl_node is None: + assert candidates[0].requires_nccl first_nccl_node = candidates[0] assert first_nccl_node is not None From 3cb1eb0ada4c27dfaa703203ab1b95cf4660bcd2 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 04:20:08 +0000 Subject: [PATCH 065/111] add asserts Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 624831999ab31..e02d3c20d28c6 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -409,6 +409,7 @@ def prepare(self): self.output_writer.start() def set_intermediate_buffer(self, data: Any): + assert self._intermediate_buffer is None self._intermediate_buffer = data def reset_intermediate_buffer(self) -> Any: @@ -420,6 +421,7 @@ def _read(self) -> bool: """ Read input data from upstream DAG nodes and cache the intermediate result. """ + assert self._intermediate_buffer is None try: res = self.input_reader.read() self.set_intermediate_buffer(res) From ed5836d3684a81ad7375b5177e36a252be7e32e0 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 04:25:10 +0000 Subject: [PATCH 066/111] update comments for actor_to_candidates Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index e02d3c20d28c6..2cb5db49af6ef 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -166,7 +166,9 @@ def _select_next_nodes( Args: actor_to_candidates: A dictionary mapping an actor id to a list of - candidate nodes with zero in-degree. + candidate nodes with zero in-degree. The list is maintained as a + priority queue, so the head of the queue, i.e., `candidates[0]`, is + the node with the smallest `bind_index`. graph: A dictionary mapping the index of a task to a dictionary of its DAGOperationGraphNodes for different operations. """ From c1becc9af8ca1cc3ae5bf8cd21029fb7cd64e16b Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 04:31:49 +0000 Subject: [PATCH 067/111] add assert, and unpack tuple Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 2cb5db49af6ef..3d4548cdc564b 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -187,13 +187,13 @@ def _select_next_nodes( first_nccl_node = candidates[0] assert first_nccl_node is not None + assert first_nccl_node.operation.type == DAGNodeOperationType.WRITE next_nodes.append( heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) ) for downstream_node_metadata in first_nccl_node.out_edges: - downstream_node = graph[downstream_node_metadata[0]][ - downstream_node_metadata[1] - ] + global_idx, op_type = downstream_node_metadata[0], downstream_node_metadata[1] + downstream_node = graph[global_idx][op_type] next_nodes.append(downstream_node) assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) return next_nodes From 255fe00dea0057cffaef2fa62fa04976e4fa3bf2 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 8 Aug 2024 04:40:33 +0000 Subject: [PATCH 068/111] rename res to input_data Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 3d4548cdc564b..82bdddb1580c1 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -425,8 +425,8 @@ def _read(self) -> bool: """ assert self._intermediate_buffer is None try: - res = self.input_reader.read() - self.set_intermediate_buffer(res) + input_data = self.input_reader.read() + self.set_intermediate_buffer(input_data) return False except RayChannelError: # Channel closed. Exit the loop. @@ -439,10 +439,10 @@ def _compute(self, class_handle) -> bool: that the last operation executed is READ so that the function retrieves the correct intermediate result. """ - res = self.reset_intermediate_buffer() + input_data = self.reset_intermediate_buffer() method = getattr(class_handle, self.method_name) try: - _process_return_vals(res, return_single_output=False) + _process_return_vals(input_data, return_single_output=False) except Exception as exc: # Previous task raised an application-level exception. # Propagate it and skip the actual task. We don't need to wrap the @@ -453,7 +453,7 @@ def _compute(self, class_handle) -> bool: resolved_inputs = [] for task_input in self.task_inputs: - resolved_inputs.append(task_input.resolve(res)) + resolved_inputs.append(task_input.resolve(input_data)) try: output_val = method(*resolved_inputs, **self.resolved_kwargs) From 6b03d0b6cc022c016f932f2e3b69b17bf102e21c Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 02:11:10 +0000 Subject: [PATCH 069/111] use 4 GPUs in CI Signed-off-by: Kai-Hsun Chen --- .buildkite/core.rayci.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 2929afca4a44d..8caa6b9e78684 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -379,7 +379,6 @@ steps: # This machine has 4 GPUs, and we need 2 GPUs, so allow 2 tests to run in # parallel. - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core - --parallelism-per-worker 2 --gpus 2 --build-name coregpubuild --only-tags multi_gpu depends_on: coregpubuild From 51c2c71c5f21b14084cf53d5c85f1875aacc8aa6 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 02:31:11 +0000 Subject: [PATCH 070/111] move _select_next_nodes to dag_node_operation.py Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 62 +---------------- python/ray/dag/dag_node_operation.py | 68 ++++++++++++++++++- .../experimental/test_execution_schedule.py | 3 +- 3 files changed, 70 insertions(+), 63 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 82bdddb1580c1..fd4adc63c5aa1 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -41,6 +41,7 @@ DAGNodeOperation, DAGNodeOperationType, DAGOperationGraphNode, + _select_next_nodes, ) from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy @@ -138,67 +139,6 @@ def _wrap_exception(exc): return wrapped -def _select_next_nodes( - actor_to_candidates: Dict["ray._raylet.ActorID", List[DAGOperationGraphNode]], - graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], -): - """ - This function selects the next nodes for topological sort to generate execution - schedule. If there are multiple DAGOperationGraphNodes with zero in-degree, - select nodes based on the following rules: - - #1 If the nodes are not NCCL write nodes, select the one with the smallest - `bind_index`. If there are multiple candidate nodes with the smallest - `bind_index` of the actors that they belong to, any one of them is - acceptable. For the implementation details, we maintain a priority queue - for each actor, where the head of the priority queue is the node with the - smallest `bind_index`. - #2 If #1 cannot be satisfied, it means that all candidate nodes are NCCL write - nodes. In this case, select the one at the head of the priority queue and - its downstream nodes, which are NCCL read nodes, regardless of whether the - downstream nodes are heads of their own priority queues. - - This function may return multiple nodes if they are NCCL nodes. In that case, - this function only removes the NCCL write node, which is also the head of a - priority queue. Other nodes will be removed in the following iterations. - Additionally, visited_nodes ensures that the same node will not be scheduled - more than once. - - Args: - actor_to_candidates: A dictionary mapping an actor id to a list of - candidate nodes with zero in-degree. The list is maintained as a - priority queue, so the head of the queue, i.e., `candidates[0]`, is - the node with the smallest `bind_index`. - graph: A dictionary mapping the index of a task to a dictionary of its - DAGOperationGraphNodes for different operations. - """ - next_nodes: List[DAGOperationGraphNode] = [] - first_nccl_node: Optional[DAGOperationGraphNode] = None - for _, candidates in actor_to_candidates.items(): - if ( - not candidates[0].requires_nccl - or candidates[0].operation.type != DAGNodeOperationType.WRITE - ): - next_nodes.append(heapq.heappop(candidates)) - assert len(next_nodes) == 1 - return next_nodes - if first_nccl_node is None: - assert candidates[0].requires_nccl - first_nccl_node = candidates[0] - - assert first_nccl_node is not None - assert first_nccl_node.operation.type == DAGNodeOperationType.WRITE - next_nodes.append( - heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) - ) - for downstream_node_metadata in first_nccl_node.out_edges: - global_idx, op_type = downstream_node_metadata[0], downstream_node_metadata[1] - downstream_node = graph[global_idx][op_type] - next_nodes.append(downstream_node) - assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) - return next_nodes - - @DeveloperAPI class CompiledTask: """Wraps the normal Ray DAGNode with some metadata.""" diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 6f47033b0cb40..c7d6358392285 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -1,8 +1,9 @@ from functools import total_ordering from enum import Enum -from typing import Set, Tuple +from typing import Set, Tuple, List, Dict, Optional from ray.util.annotations import DeveloperAPI import ray +import heapq @DeveloperAPI @@ -101,3 +102,68 @@ def add_edge(self, out_node: "DAGOperationGraphNode"): """ self.out_edges.add((out_node.idx, out_node.operation.type)) out_node.in_edges.add((self.idx, self.operation.type)) + + +def _select_next_nodes( + actor_to_candidates: Dict["ray._raylet.ActorID", List[DAGOperationGraphNode]], + graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], +): + """ + This function selects the next nodes for topological sort to generate execution + schedule. If there are multiple DAGOperationGraphNodes with zero in-degree, + select nodes based on the following rules: + + #1 If the nodes are not NCCL write nodes, select the one with the smallest + `bind_index`. If there are multiple candidate nodes with the smallest + `bind_index` of the actors that they belong to, any one of them is + acceptable. For the implementation details, we maintain a priority queue + for each actor, where the head of the priority queue is the node with the + smallest `bind_index`. + #2 If #1 cannot be satisfied, it means that all candidate nodes are NCCL write + nodes. In this case, select the one at the head of the priority queue and + its downstream nodes, which are NCCL read nodes, regardless of whether the + downstream nodes are heads of their own priority queues. + + This function may return multiple nodes if they are NCCL nodes. In that case, + this function only removes the NCCL write node, which is also the head of a + priority queue. Other nodes will be removed in the following iterations. + Additionally, visited_nodes ensures that the same node will not be scheduled + more than once. + + Args: + actor_to_candidates: A dictionary mapping an actor id to a list of + candidate nodes with zero in-degree. The list is maintained as a + priority queue, so the head of the queue, i.e., `candidates[0]`, is + the node with the smallest `bind_index`. + graph: A dictionary mapping the index of a task to a dictionary of its + DAGOperationGraphNodes for different operations. + + Returns: + A list of DAGOperationGraphNodes to be placed into the corresponding + execution schedules. + """ + next_nodes: List[DAGOperationGraphNode] = [] + first_nccl_node: Optional[DAGOperationGraphNode] = None + for _, candidates in actor_to_candidates.items(): + if ( + not candidates[0].requires_nccl + or candidates[0].operation.type != DAGNodeOperationType.WRITE + ): + next_nodes.append(heapq.heappop(candidates)) + assert len(next_nodes) == 1 + return next_nodes + if first_nccl_node is None: + assert candidates[0].requires_nccl + first_nccl_node = candidates[0] + + assert first_nccl_node is not None + assert first_nccl_node.operation.type == DAGNodeOperationType.WRITE + next_nodes.append( + heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) + ) + for downstream_node_metadata in first_nccl_node.out_edges: + global_idx, op_type = downstream_node_metadata[0], downstream_node_metadata[1] + downstream_node = graph[global_idx][op_type] + next_nodes.append(downstream_node) + assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) + return next_nodes diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 2d745df489cf3..aada9791dadf6 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -13,8 +13,9 @@ DAGNodeOperationType, DAGOperationGraphNode, DAGNodeOperation, + _select_next_nodes, ) -from ray.dag.compiled_dag_node import _select_next_nodes, CompiledDAG, CompiledTask +from ray.dag.compiled_dag_node import CompiledDAG, CompiledTask import torch from typing import List, Dict, Tuple from dataclasses import dataclass, field From eaa3a4da4a702ae60ccf57f5f4bda79e3033d0d7 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 02:33:19 +0000 Subject: [PATCH 071/111] update comments for _select_next_nodes Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index c7d6358392285..d84de06b16caf 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -110,8 +110,8 @@ def _select_next_nodes( ): """ This function selects the next nodes for topological sort to generate execution - schedule. If there are multiple DAGOperationGraphNodes with zero in-degree, - select nodes based on the following rules: + schedule. If there are multiple candidate DAGOperationGraphNodes, select nodes + based on the following rules: #1 If the nodes are not NCCL write nodes, select the one with the smallest `bind_index`. If there are multiple candidate nodes with the smallest @@ -132,9 +132,9 @@ def _select_next_nodes( Args: actor_to_candidates: A dictionary mapping an actor id to a list of - candidate nodes with zero in-degree. The list is maintained as a - priority queue, so the head of the queue, i.e., `candidates[0]`, is - the node with the smallest `bind_index`. + candidate nodes. The list is maintained as a priority queue, so + the head of the queue, i.e., `candidates[0]`, is the node with + the smallest `bind_index`. graph: A dictionary mapping the index of a task to a dictionary of its DAGOperationGraphNodes for different operations. From 45d9f43dde39d7c91fd6cd862da951062a282496 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 02:37:31 +0000 Subject: [PATCH 072/111] update comments Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index d84de06b16caf..66f573e7486cb 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -121,14 +121,12 @@ def _select_next_nodes( smallest `bind_index`. #2 If #1 cannot be satisfied, it means that all candidate nodes are NCCL write nodes. In this case, select the one at the head of the priority queue and - its downstream nodes, which are NCCL read nodes, regardless of whether the - downstream nodes are heads of their own priority queues. + its immediately downstream nodes, which are NCCL read nodes, regardless of + whether the downstream nodes are heads of their own priority queues. This function may return multiple nodes if they are NCCL nodes. In that case, this function only removes the NCCL write node, which is also the head of a priority queue. Other nodes will be removed in the following iterations. - Additionally, visited_nodes ensures that the same node will not be scheduled - more than once. Args: actor_to_candidates: A dictionary mapping an actor id to a list of From 52c6f3632030cac1668988073b0fe0d2798a0592 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 02:55:51 +0000 Subject: [PATCH 073/111] remove DAGNodeOperationType's DeveloperAPI Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 32 +-- python/ray/dag/dag_node_operation.py | 15 +- .../experimental/test_execution_schedule.py | 228 +++++++++--------- 3 files changed, 137 insertions(+), 138 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index fd4adc63c5aa1..2a6f1608c2d47 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -39,7 +39,7 @@ from ray.dag.dag_node_operation import ( DAGNodeOperation, - DAGNodeOperationType, + _DAGNodeOperationType, DAGOperationGraphNode, _select_next_nodes, ) @@ -416,7 +416,7 @@ def _write(self) -> bool: # Channel closed. Exit the loop. return True - def exec_operation(self, class_handle, op_type: DAGNodeOperationType): + def exec_operation(self, class_handle, op_type: _DAGNodeOperationType): """ An ExecutableTask corresponds to a DAGNode. It consists of three operations: READ, COMPUTE, and WRITE, which should be executed in @@ -431,11 +431,11 @@ def exec_operation(self, class_handle, op_type: DAGNodeOperationType): Returns: True if the next operation should not be executed; otherwise, False. """ - if op_type == DAGNodeOperationType.READ: + if op_type == _DAGNodeOperationType.READ: return self._read() - elif op_type == DAGNodeOperationType.COMPUTE: + elif op_type == _DAGNodeOperationType.COMPUTE: return self._compute(class_handle) - elif op_type == DAGNodeOperationType.WRITE: + elif op_type == _DAGNodeOperationType.WRITE: return self._write() @@ -1163,19 +1163,19 @@ def _generate_dag_operation_graph_node( requires_nccl = dag_node.type_hint.requires_nccl() read_node = DAGOperationGraphNode( - DAGNodeOperation(local_idx, DAGNodeOperationType.READ), + DAGNodeOperation(local_idx, _DAGNodeOperationType.READ), idx, actor_handle, requires_nccl, ) compute_node = DAGOperationGraphNode( - DAGNodeOperation(local_idx, DAGNodeOperationType.COMPUTE), + DAGNodeOperation(local_idx, _DAGNodeOperationType.COMPUTE), idx, actor_handle, requires_nccl, ) write_node = DAGOperationGraphNode( - DAGNodeOperation(local_idx, DAGNodeOperationType.WRITE), + DAGNodeOperation(local_idx, _DAGNodeOperationType.WRITE), idx, actor_handle, requires_nccl, @@ -1190,7 +1190,7 @@ def _build_dag_node_operation_graph( actor_to_operation_nodes: Dict[ "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] ], - ) -> Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]]: + ) -> Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]]: """ Generate a DAG node operation graph by adding edges based on the following rules: @@ -1213,12 +1213,12 @@ def _build_dag_node_operation_graph( Returns: A graph where each node is a DAGOperationGraphNode. The key is the index of the task in idx_to_task, and the value is a dictionary that maps the - DAGNodeOperationType (READ, COMPUTE, or WRITE) to the corresponding + _DAGNodeOperationType (READ, COMPUTE, or WRITE) to the corresponding DAGOperationGraphNode. """ assert self.idx_to_task - graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]] = {} + graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]] = {} for _, operation_nodes_list in actor_to_operation_nodes.items(): prev_compute_node = None @@ -1240,9 +1240,9 @@ def _build_dag_node_operation_graph( prev_compute_node = compute_node assert idx not in graph graph[idx] = { - DAGNodeOperationType.READ: read_node, - DAGNodeOperationType.COMPUTE: compute_node, - DAGNodeOperationType.WRITE: write_node, + _DAGNodeOperationType.READ: read_node, + _DAGNodeOperationType.COMPUTE: compute_node, + _DAGNodeOperationType.WRITE: write_node, } from ray.dag import ( @@ -1258,8 +1258,8 @@ def _build_dag_node_operation_graph( downstream_dag_node = self.idx_to_task[downstream_idx].dag_node if isinstance(downstream_dag_node, MultiOutputNode): continue - graph[idx][DAGNodeOperationType.WRITE].add_edge( - graph[downstream_idx][DAGNodeOperationType.READ] + graph[idx][_DAGNodeOperationType.WRITE].add_edge( + graph[downstream_idx][_DAGNodeOperationType.READ] ) return graph diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 66f573e7486cb..382b4fcc309c6 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -6,8 +6,7 @@ import heapq -@DeveloperAPI -class DAGNodeOperationType(Enum): +class _DAGNodeOperationType(Enum): """ There are three types of operations that a DAG node can perform: 1. READ: Read from an input channel. @@ -25,7 +24,7 @@ class DAGNodeOperation: def __init__( self, idx: int, - operation_type: DAGNodeOperationType, + operation_type: _DAGNodeOperationType, ): """ Args: @@ -64,8 +63,8 @@ def __init__( self.idx = idx self.actor_handle = actor_handle self.requires_nccl = requires_nccl - self.in_edges: Set[Tuple[int, DAGNodeOperationType]] = set() - self.out_edges: Set[Tuple[int, DAGNodeOperationType]] = set() + self.in_edges: Set[Tuple[int, _DAGNodeOperationType]] = set() + self.out_edges: Set[Tuple[int, _DAGNodeOperationType]] = set() @property def in_degree(self) -> int: @@ -106,7 +105,7 @@ def add_edge(self, out_node: "DAGOperationGraphNode"): def _select_next_nodes( actor_to_candidates: Dict["ray._raylet.ActorID", List[DAGOperationGraphNode]], - graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], ): """ This function selects the next nodes for topological sort to generate execution @@ -145,7 +144,7 @@ def _select_next_nodes( for _, candidates in actor_to_candidates.items(): if ( not candidates[0].requires_nccl - or candidates[0].operation.type != DAGNodeOperationType.WRITE + or candidates[0].operation.type != _DAGNodeOperationType.WRITE ): next_nodes.append(heapq.heappop(candidates)) assert len(next_nodes) == 1 @@ -155,7 +154,7 @@ def _select_next_nodes( first_nccl_node = candidates[0] assert first_nccl_node is not None - assert first_nccl_node.operation.type == DAGNodeOperationType.WRITE + assert first_nccl_node.operation.type == _DAGNodeOperationType.WRITE next_nodes.append( heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) ) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index aada9791dadf6..1a10750eba356 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -10,7 +10,7 @@ from ray.tests.conftest import * # noqa from ray.dag import InputNode, MultiOutputNode, ClassMethodNode from ray.dag.dag_node_operation import ( - DAGNodeOperationType, + _DAGNodeOperationType, DAGOperationGraphNode, DAGNodeOperation, _select_next_nodes, @@ -303,35 +303,35 @@ def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): compiled_dag = dag.experimental_compile() w1_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (0, DAGNodeOperationType.WRITE), - (1, DAGNodeOperationType.READ), - (1, DAGNodeOperationType.COMPUTE), - (1, DAGNodeOperationType.WRITE), - (2, DAGNodeOperationType.READ), - (2, DAGNodeOperationType.COMPUTE), - (2, DAGNodeOperationType.WRITE), - (3, DAGNodeOperationType.READ), - (3, DAGNodeOperationType.COMPUTE), - (3, DAGNodeOperationType.WRITE), - (4, DAGNodeOperationType.READ), - (4, DAGNodeOperationType.COMPUTE), - (4, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), + (2, _DAGNodeOperationType.READ), + (2, _DAGNodeOperationType.COMPUTE), + (2, _DAGNodeOperationType.WRITE), + (3, _DAGNodeOperationType.READ), + (3, _DAGNodeOperationType.COMPUTE), + (3, _DAGNodeOperationType.WRITE), + (4, _DAGNodeOperationType.READ), + (4, _DAGNodeOperationType.COMPUTE), + (4, _DAGNodeOperationType.WRITE), ] w2_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (0, DAGNodeOperationType.WRITE), - (1, DAGNodeOperationType.READ), - (1, DAGNodeOperationType.COMPUTE), - (2, DAGNodeOperationType.READ), - (1, DAGNodeOperationType.WRITE), - (2, DAGNodeOperationType.COMPUTE), - (2, DAGNodeOperationType.WRITE), - (3, DAGNodeOperationType.READ), - (3, DAGNodeOperationType.COMPUTE), - (3, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (2, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.WRITE), + (2, _DAGNodeOperationType.COMPUTE), + (2, _DAGNodeOperationType.WRITE), + (3, _DAGNodeOperationType.READ), + (3, _DAGNodeOperationType.COMPUTE), + (3, _DAGNodeOperationType.WRITE), ] w1_schedule = compiled_dag.actor_to_execution_schedule[w1] w2_schedule = compiled_dag.actor_to_execution_schedule[w2] @@ -409,22 +409,22 @@ def test_three_actors_with_nccl_1(ray_start_regular): compiled_dag = dag.experimental_compile() a_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (0, DAGNodeOperationType.WRITE), - (1, DAGNodeOperationType.READ), - (1, DAGNodeOperationType.COMPUTE), - (1, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), ] b_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (0, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), ] c_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (0, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), ] a_schedule = compiled_dag.actor_to_execution_schedule[a] b_schedule = compiled_dag.actor_to_execution_schedule[b] @@ -480,28 +480,28 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): compiled_dag = dag.experimental_compile() a_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (1, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.WRITE), - (1, DAGNodeOperationType.COMPUTE), - (1, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), ] b_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (1, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.WRITE), - (1, DAGNodeOperationType.COMPUTE), - (1, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), ] c_expected_schedule = [ - (0, DAGNodeOperationType.READ), - (0, DAGNodeOperationType.COMPUTE), - (0, DAGNodeOperationType.WRITE), - (1, DAGNodeOperationType.READ), - (1, DAGNodeOperationType.COMPUTE), - (1, DAGNodeOperationType.WRITE), + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), ] a_schedule = compiled_dag.actor_to_execution_schedule[a] @@ -531,7 +531,7 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): def generate_dag_graph_nodes(local_idx, global_idx, actor_handle, requires_nccl): graph_nodes = {} - for op_type in DAGNodeOperationType: + for op_type in _DAGNodeOperationType: graph_nodes[op_type] = DAGOperationGraphNode( DAGNodeOperation(local_idx, op_type), global_idx, @@ -568,7 +568,7 @@ def test_two_candidates_on_same_actor(self): # actor's `executable_tasks` list is 0. global_idx_1 = 1 dag_node_1 = DAGOperationGraphNode( - DAGNodeOperation(0, DAGNodeOperationType.READ), + DAGNodeOperation(0, _DAGNodeOperationType.READ), global_idx_1, fake_actor, False, @@ -577,7 +577,7 @@ def test_two_candidates_on_same_actor(self): # actor's `executable_tasks` list is 1. global_idx_2 = 2 dag_node_2 = DAGOperationGraphNode( - DAGNodeOperation(1, DAGNodeOperationType.READ), + DAGNodeOperation(1, _DAGNodeOperationType.READ), global_idx_2, fake_actor, False, @@ -616,24 +616,24 @@ def test_only_one_nccl_write(self, monkeypatch): local_idx_2, global_idx_2, fake_actor_2, False ), } - del mock_graph[global_idx_1][DAGNodeOperationType.READ] - del mock_graph[global_idx_1][DAGNodeOperationType.COMPUTE] - mock_graph[global_idx_1][DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_2][DAGNodeOperationType.READ] + del mock_graph[global_idx_1][_DAGNodeOperationType.READ] + del mock_graph[global_idx_1][_DAGNodeOperationType.COMPUTE] + mock_graph[global_idx_1][_DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_2][_DAGNodeOperationType.READ] ) - mock_graph[global_idx_2][DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_2][DAGNodeOperationType.COMPUTE] + mock_graph[global_idx_2][_DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE] ) - mock_graph[global_idx_2][DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_2][DAGNodeOperationType.WRITE] + mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_2][_DAGNodeOperationType.WRITE] ) mock_actor_to_candidates = { - fake_actor_1: [mock_graph[global_idx_1][DAGNodeOperationType.WRITE]], + fake_actor_1: [mock_graph[global_idx_1][_DAGNodeOperationType.WRITE]], } next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) assert len(next_nodes) == 2 - assert next_nodes[0] == mock_graph[global_idx_1][DAGNodeOperationType.WRITE] - assert next_nodes[1] == mock_graph[global_idx_2][DAGNodeOperationType.READ] + assert next_nodes[0] == mock_graph[global_idx_1][_DAGNodeOperationType.WRITE] + assert next_nodes[1] == mock_graph[global_idx_2][_DAGNodeOperationType.READ] def test_two_nccl_writes(self, monkeypatch): """ @@ -672,47 +672,47 @@ def test_two_nccl_writes(self, monkeypatch): local_idx_2_1, global_idx_2_1, fake_actor_2, False ), } - del mock_graph[global_idx_1_0][DAGNodeOperationType.READ] - del mock_graph[global_idx_1_0][DAGNodeOperationType.COMPUTE] - del mock_graph[global_idx_2_0][DAGNodeOperationType.READ] - del mock_graph[global_idx_2_0][DAGNodeOperationType.COMPUTE] + del mock_graph[global_idx_1_0][_DAGNodeOperationType.READ] + del mock_graph[global_idx_1_0][_DAGNodeOperationType.COMPUTE] + del mock_graph[global_idx_2_0][_DAGNodeOperationType.READ] + del mock_graph[global_idx_2_0][_DAGNodeOperationType.COMPUTE] - mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_2_1][DAGNodeOperationType.READ] + mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] ) - mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_1_1][DAGNodeOperationType.READ] + mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.READ] ) - mock_graph[global_idx_2_1][DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_2_1][DAGNodeOperationType.COMPUTE] + mock_graph[global_idx_2_1][_DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE] ) - mock_graph[global_idx_2_1][DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_2_1][DAGNodeOperationType.WRITE] + mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.WRITE] ) - mock_graph[global_idx_1_1][DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_1_1][DAGNodeOperationType.COMPUTE] + mock_graph[global_idx_1_1][_DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE] ) - mock_graph[global_idx_1_1][DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_1_1][DAGNodeOperationType.WRITE] + mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.WRITE] ) mock_actor_to_candidates = { - fake_actor_1: [mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE]], - fake_actor_2: [mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE]], + fake_actor_1: [mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE]], + fake_actor_2: [mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE]], } next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) assert len(next_nodes) == 2 assert next_nodes[0] in [ - mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE], - mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE], + mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE], + mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE], ] - if next_nodes[0] == mock_graph[global_idx_1_0][DAGNodeOperationType.WRITE]: + if next_nodes[0] == mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE]: assert ( - next_nodes[1] == mock_graph[global_idx_2_1][DAGNodeOperationType.READ] + next_nodes[1] == mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] ) - elif next_nodes[0] == mock_graph[global_idx_2_0][DAGNodeOperationType.WRITE]: + elif next_nodes[0] == mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE]: assert ( - next_nodes[1] == mock_graph[global_idx_1_1][DAGNodeOperationType.READ] + next_nodes[1] == mock_graph[global_idx_1_1][_DAGNodeOperationType.READ] ) @@ -725,7 +725,7 @@ class TestBuildDAGNodeOperationGraph: def check_edges_between_read_compute_write( self, - graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], global_idx: int, expected_num_edges: List[Tuple[int, int]], ): @@ -743,22 +743,22 @@ def check_edges_between_read_compute_write( """ assert len(expected_num_edges) == 3 assert len(graph[global_idx]) == 3 - read_node = graph[global_idx][DAGNodeOperationType.READ] - compute_node = graph[global_idx][DAGNodeOperationType.COMPUTE] - write_node = graph[global_idx][DAGNodeOperationType.WRITE] + read_node = graph[global_idx][_DAGNodeOperationType.READ] + compute_node = graph[global_idx][_DAGNodeOperationType.COMPUTE] + write_node = graph[global_idx][_DAGNodeOperationType.WRITE] for idx, node in enumerate([read_node, compute_node, write_node]): assert node.in_degree == expected_num_edges[idx][0] assert len(node.out_edges) == expected_num_edges[idx][1] - assert (global_idx, DAGNodeOperationType.COMPUTE) in read_node.out_edges - assert (global_idx, DAGNodeOperationType.READ) in compute_node.in_edges - assert (global_idx, DAGNodeOperationType.WRITE) in compute_node.out_edges - assert (global_idx, DAGNodeOperationType.COMPUTE) in write_node.in_edges + assert (global_idx, _DAGNodeOperationType.COMPUTE) in read_node.out_edges + assert (global_idx, _DAGNodeOperationType.READ) in compute_node.in_edges + assert (global_idx, _DAGNodeOperationType.WRITE) in compute_node.out_edges + assert (global_idx, _DAGNodeOperationType.COMPUTE) in write_node.in_edges def check_edge_between_writer_and_reader( self, - graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], writer_global_idx: int, reader_global_idx: int, ): @@ -772,15 +772,15 @@ def check_edge_between_writer_and_reader( reader_global_idx: The global index of the task used to access the task that the reader belongs to in `idx_to_task`. """ - write_node = graph[writer_global_idx][DAGNodeOperationType.WRITE] - read_node = graph[reader_global_idx][DAGNodeOperationType.READ] + write_node = graph[writer_global_idx][_DAGNodeOperationType.WRITE] + read_node = graph[reader_global_idx][_DAGNodeOperationType.READ] - assert (reader_global_idx, DAGNodeOperationType.READ) in write_node.out_edges - assert (writer_global_idx, DAGNodeOperationType.WRITE) in read_node.in_edges + assert (reader_global_idx, _DAGNodeOperationType.READ) in write_node.out_edges + assert (writer_global_idx, _DAGNodeOperationType.WRITE) in read_node.in_edges def check_edge_between_compute_nodes( self, - graph: Dict[int, Dict[DAGNodeOperationType, DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], global_idx_1: int, global_idx_2: int, ): @@ -797,11 +797,11 @@ def check_edge_between_compute_nodes( `bind_index` of the second task is equal to the `bind_index` of the first task plus one. """ - compute_node_1 = graph[global_idx_1][DAGNodeOperationType.COMPUTE] - compute_node_2 = graph[global_idx_2][DAGNodeOperationType.COMPUTE] + compute_node_1 = graph[global_idx_1][_DAGNodeOperationType.COMPUTE] + compute_node_2 = graph[global_idx_2][_DAGNodeOperationType.COMPUTE] - assert (global_idx_2, DAGNodeOperationType.COMPUTE) in compute_node_1.out_edges - assert (global_idx_1, DAGNodeOperationType.COMPUTE) in compute_node_2.in_edges + assert (global_idx_2, _DAGNodeOperationType.COMPUTE) in compute_node_1.out_edges + assert (global_idx_1, _DAGNodeOperationType.COMPUTE) in compute_node_2.in_edges def test_edges_between_read_compute_write(self, monkeypatch): """ From ea8def23065f5dba76a72debb8692557e02afc79 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 04:45:51 +0000 Subject: [PATCH 074/111] remove double negative Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 382b4fcc309c6..75f3bdccc72d4 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -142,9 +142,9 @@ def _select_next_nodes( next_nodes: List[DAGOperationGraphNode] = [] first_nccl_node: Optional[DAGOperationGraphNode] = None for _, candidates in actor_to_candidates.items(): - if ( - not candidates[0].requires_nccl - or candidates[0].operation.type != _DAGNodeOperationType.WRITE + if not ( + candidates[0].requires_nccl + and candidates[0].operation.type == _DAGNodeOperationType.WRITE ): next_nodes.append(heapq.heappop(candidates)) assert len(next_nodes) == 1 From 9a81203c1f3ff495d5767cf3ff66a1742e51337d Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 04:53:30 +0000 Subject: [PATCH 075/111] move first_nccl_node into a separate loop Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 75f3bdccc72d4..4af2918396f5b 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -140,7 +140,6 @@ def _select_next_nodes( execution schedules. """ next_nodes: List[DAGOperationGraphNode] = [] - first_nccl_node: Optional[DAGOperationGraphNode] = None for _, candidates in actor_to_candidates.items(): if not ( candidates[0].requires_nccl @@ -149,12 +148,17 @@ def _select_next_nodes( next_nodes.append(heapq.heappop(candidates)) assert len(next_nodes) == 1 return next_nodes - if first_nccl_node is None: - assert candidates[0].requires_nccl + + first_nccl_node: Optional[DAGOperationGraphNode] = None + for _, candidates in actor_to_candidates.items(): + if ( + candidates[0].requires_nccl + and candidates[0].operation.type == _DAGNodeOperationType.WRITE + ): first_nccl_node = candidates[0] + break assert first_nccl_node is not None - assert first_nccl_node.operation.type == _DAGNodeOperationType.WRITE next_nodes.append( heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) ) From dc460135b64db8d982b5686c47390d742ed7eb53 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 04:59:09 +0000 Subject: [PATCH 076/111] add a comment for picking nccl write/read nodes Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 4af2918396f5b..711a4d6294310 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -162,9 +162,13 @@ def _select_next_nodes( next_nodes.append( heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) ) + + # An NCCL write node is picked. NCCL is a blocking operation, so we need to pick all + # the corresponding NCCL read nodes to avoid a deadlock. for downstream_node_metadata in first_nccl_node.out_edges: global_idx, op_type = downstream_node_metadata[0], downstream_node_metadata[1] downstream_node = graph[global_idx][op_type] + assert downstream_node.operation.type == _DAGNodeOperationType.READ next_nodes.append(downstream_node) assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) return next_nodes From 18b773a84704822da3a10c4d00416e3d7fdd13be Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 05:11:23 +0000 Subject: [PATCH 077/111] move return outside try-except Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 2a6f1608c2d47..2ca68ac55c16a 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -362,15 +362,19 @@ def reset_intermediate_buffer(self) -> Any: def _read(self) -> bool: """ Read input data from upstream DAG nodes and cache the intermediate result. + + Returns: + True if system error occurs and exit the loop; otherwise, False. """ assert self._intermediate_buffer is None + exit = False try: input_data = self.input_reader.read() self.set_intermediate_buffer(input_data) - return False except RayChannelError: # Channel closed. Exit the loop. - return True + exit = True + return exit def _compute(self, class_handle) -> bool: """ @@ -378,6 +382,9 @@ def _compute(self, class_handle) -> bool: computation. Then, cache the new intermediate result. The caller must ensure that the last operation executed is READ so that the function retrieves the correct intermediate result. + + Returns: + True if system error occurs and exit the loop; otherwise, False. """ input_data = self.reset_intermediate_buffer() method = getattr(class_handle, self.method_name) @@ -407,16 +414,20 @@ def _write(self) -> bool: Retrieve the intermediate result from the COMPUTE operation and write to its downstream DAG nodes. The caller must ensure that the last operation executed is COMPUTE so that the function retrieves the correct intermediate result. + + Returns: + True if system error occurs and exit the loop; otherwise, False. """ output_val = self.reset_intermediate_buffer() + exit = False try: self.output_writer.write(output_val) - return False except RayChannelError: # Channel closed. Exit the loop. - return True + exit = True + return exit - def exec_operation(self, class_handle, op_type: _DAGNodeOperationType): + def exec_operation(self, class_handle, op_type: _DAGNodeOperationType) -> bool: """ An ExecutableTask corresponds to a DAGNode. It consists of three operations: READ, COMPUTE, and WRITE, which should be executed in From f51bd1a9d48c296c89c729e9756e5da37cd57e1b Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 05:24:15 +0000 Subject: [PATCH 078/111] add comments for _build_execution_schedule Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 2ca68ac55c16a..b27164aa13a06 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1301,11 +1301,17 @@ def _build_execution_schedule(self): actor_to_operation_nodes = self._generate_dag_operation_graph_node() graph = self._build_dag_node_operation_graph(actor_to_operation_nodes) + # A dictionary mapping an actor id to a list of candidate nodes. The list + # is maintained as a priority queue, so the head of the queue, i.e., + # `candidates[0]`, is the node with the smallest `bind_index`. actor_to_candidates: Dict[ "ray._raylet.ActorID", List[DAGOperationGraphNode] ] = defaultdict(list) for _, node_dict in graph.items(): for _, node in node_dict.items(): + # A node with a zero in-degree edge means all of its dependencies + # have been satisfied, including both data and control dependencies. + # Therefore, it is a candidate for execution. if node.in_degree == 0: heapq.heappush( actor_to_candidates[node.actor_handle._actor_id], node From fc123cbb610f198a117f722d65013ec752e0047c Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 05:42:35 +0000 Subject: [PATCH 079/111] add test to ensure _select_next_nodes is deterministic Signed-off-by: Kai-Hsun Chen --- .../experimental/test_execution_schedule.py | 98 +++++++++---------- 1 file changed, 48 insertions(+), 50 deletions(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 1a10750eba356..a4a14e2f8f304 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -658,61 +658,59 @@ def test_two_nccl_writes(self, monkeypatch): fake_actor_2 = ActorHandle("fake_actor_2") global_idx_2_0, local_idx_2_0 = 2, 0 global_idx_2_1, local_idx_2_1 = 4, 1 - mock_graph = { - global_idx_1_0: generate_dag_graph_nodes( - local_idx_1_0, global_idx_1_0, fake_actor_1, True - ), - global_idx_1_1: generate_dag_graph_nodes( - local_idx_1_1, global_idx_1_1, fake_actor_1, False - ), - global_idx_2_0: generate_dag_graph_nodes( - local_idx_2_0, global_idx_2_0, fake_actor_2, True - ), - global_idx_2_1: generate_dag_graph_nodes( - local_idx_2_1, global_idx_2_1, fake_actor_2, False - ), - } - del mock_graph[global_idx_1_0][_DAGNodeOperationType.READ] - del mock_graph[global_idx_1_0][_DAGNodeOperationType.COMPUTE] - del mock_graph[global_idx_2_0][_DAGNodeOperationType.READ] - del mock_graph[global_idx_2_0][_DAGNodeOperationType.COMPUTE] - mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] - ) - mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.READ] - ) - mock_graph[global_idx_2_1][_DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE] - ) - mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.WRITE] - ) - mock_graph[global_idx_1_1][_DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE] - ) - mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.WRITE] - ) - mock_actor_to_candidates = { - fake_actor_1: [mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE]], - fake_actor_2: [mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE]], - } + # Run the test 10 times to ensure that the result of `_select_next_nodes` + # is deterministic. + for _ in range(20): + mock_graph = { + global_idx_1_0: generate_dag_graph_nodes( + local_idx_1_0, global_idx_1_0, fake_actor_1, True + ), + global_idx_1_1: generate_dag_graph_nodes( + local_idx_1_1, global_idx_1_1, fake_actor_1, False + ), + global_idx_2_0: generate_dag_graph_nodes( + local_idx_2_0, global_idx_2_0, fake_actor_2, True + ), + global_idx_2_1: generate_dag_graph_nodes( + local_idx_2_1, global_idx_2_1, fake_actor_2, False + ), + } + del mock_graph[global_idx_1_0][_DAGNodeOperationType.READ] + del mock_graph[global_idx_1_0][_DAGNodeOperationType.COMPUTE] + del mock_graph[global_idx_2_0][_DAGNodeOperationType.READ] + del mock_graph[global_idx_2_0][_DAGNodeOperationType.COMPUTE] + + mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] + ) + mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE].add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.READ] + ) + mock_graph[global_idx_2_1][_DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE] + ) + mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.WRITE] + ) + mock_graph[global_idx_1_1][_DAGNodeOperationType.READ].add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE] + ) + mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE].add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.WRITE] + ) + mock_actor_to_candidates = { + fake_actor_1: [mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE]], + fake_actor_2: [mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE]], + } - next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) - assert len(next_nodes) == 2 - assert next_nodes[0] in [ - mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE], - mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE], - ] - if next_nodes[0] == mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE]: + next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) + assert len(next_nodes) == 2 assert ( - next_nodes[1] == mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] + next_nodes[0] == mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE] ) - elif next_nodes[0] == mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE]: assert ( - next_nodes[1] == mock_graph[global_idx_1_1][_DAGNodeOperationType.READ] + next_nodes[1] == mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] ) From a41b9a6eba532af515a9cea141558bc54481cfa7 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 06:02:19 +0000 Subject: [PATCH 080/111] add comments for class_handle Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index b27164aa13a06..a69925cc92ff5 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -383,6 +383,11 @@ def _compute(self, class_handle) -> bool: that the last operation executed is READ so that the function retrieves the correct intermediate result. + Args: + class_handle: An instance of the class to which the actor belongs. For example, + the type of `class_handle` is if the actor belongs to + the `class Worker` class. + Returns: True if system error occurs and exit the loop; otherwise, False. """ From bb6db7075c85f1a009f197b430155b91204fc04a Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 07:28:00 +0000 Subject: [PATCH 081/111] add comment for deterministism --- python/ray/dag/compiled_dag_node.py | 6 +++--- python/ray/dag/dag_node_operation.py | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index a69925cc92ff5..c1eb7e3ce6588 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -384,9 +384,9 @@ def _compute(self, class_handle) -> bool: correct intermediate result. Args: - class_handle: An instance of the class to which the actor belongs. For example, - the type of `class_handle` is if the actor belongs to - the `class Worker` class. + class_handle: An instance of the class to which the actor belongs. For + example, the type of `class_handle` is if the + actor belongs to the `class Worker` class. Returns: True if system error occurs and exit the loop; otherwise, False. diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 711a4d6294310..4df26a44e6ca5 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -114,10 +114,10 @@ def _select_next_nodes( #1 If the nodes are not NCCL write nodes, select the one with the smallest `bind_index`. If there are multiple candidate nodes with the smallest - `bind_index` of the actors that they belong to, any one of them is - acceptable. For the implementation details, we maintain a priority queue - for each actor, where the head of the priority queue is the node with the - smallest `bind_index`. + `bind_index` among the actors to which they belong, any one of them is + acceptable, but the implementation ensures the result is deterministic. + For the implementation details, we maintain a priority queue for each actor, + where the head of the priority queue is the node with the smallest `bind_index`. #2 If #1 cannot be satisfied, it means that all candidate nodes are NCCL write nodes. In this case, select the one at the head of the priority queue and its immediately downstream nodes, which are NCCL read nodes, regardless of From a835314cc22ba5461dad38c4ef2a51f9852d8136 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 08:12:07 +0000 Subject: [PATCH 082/111] add comment --- python/ray/dag/compiled_dag_node.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index c1eb7e3ce6588..adaeeb92abb7d 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1099,6 +1099,7 @@ def _get_or_compile( # Sort executable tasks based on their bind index, i.e., submission order # so that they will be executed in that order. executable_tasks.sort(key=lambda task: task.bind_index) + print(executable_tasks) self.actor_to_executable_tasks[actor_handle] = executable_tasks # Build an execution schedule for each actor @@ -1269,6 +1270,9 @@ def _build_dag_node_operation_graph( # Add an edge from WRITE of the writer task to READ of the reader task. for idx, task in self.idx_to_task.items(): if not isinstance(task.dag_node, ClassMethodNode): + # The graph is used to generate an execution schedule for each actor. + # The edge from the InputNode has no impact on the final execution + # schedule. continue for downstream_idx in task.downstream_node_idxs: downstream_dag_node = self.idx_to_task[downstream_idx].dag_node From 0893e990f623f092cf4e590d497599e7ed08a3ac Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 15:34:20 +0000 Subject: [PATCH 083/111] return actor_to_execution_schedule --- python/ray/dag/compiled_dag_node.py | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index adaeeb92abb7d..238e476aa9f8c 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1103,7 +1103,7 @@ def _get_or_compile( self.actor_to_executable_tasks[actor_handle] = executable_tasks # Build an execution schedule for each actor - self._build_execution_schedule() + self.actor_to_execution_schedule = self._build_execution_schedule() for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): self.worker_task_refs[actor_handle] = actor_handle.__ray_call__.options( concurrency_group="_ray_system" @@ -1305,7 +1305,17 @@ def _build_execution_schedule(self): [Example]: See `test_execution_schedule` for more examples. + + Returns: + actor_to_execution_schedule: A dictionary that maps an actor handle to + the execution schedule which is a list of operations to be executed. """ + # Mapping from the actor handle to the execution schedule which is a list + # of operations to be executed. + actor_to_execution_schedule: Dict[ + "ray.actor.ActorHandle", List[DAGNodeOperation] + ] = defaultdict(list) + # Step 1: Build a graph of DAGOperationGraphNode actor_to_operation_nodes = self._generate_dag_operation_graph_node() graph = self._build_dag_node_operation_graph(actor_to_operation_nodes) @@ -1334,9 +1344,7 @@ def _build_execution_schedule(self): for node in nodes: if node in visited_nodes: continue - self.actor_to_execution_schedule[node.actor_handle].append( - node.operation - ) + actor_to_execution_schedule[node.actor_handle].append(node.operation) visited_nodes.add(node) for out_node_idx, out_node_type in node.out_edges: out_node = graph[out_node_idx][out_node_type] @@ -1353,6 +1361,7 @@ def _build_execution_schedule(self): delete_keys.append(actor_id) for key in delete_keys: del actor_to_candidates[key] + return actor_to_execution_schedule def _detect_deadlock(self) -> bool: """ From d7bda839575df05aa8fca0db7294168237ba0de0 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 16:18:15 +0000 Subject: [PATCH 084/111] move _build_dag_node_operation_graph to dag_node_operation.py --- python/ray/dag/compiled_dag_node.py | 86 +------------------ python/ray/dag/dag_node_operation.py | 83 ++++++++++++++++++ .../experimental/test_execution_schedule.py | 37 +++----- 3 files changed, 101 insertions(+), 105 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 238e476aa9f8c..451b0662a1fbe 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -42,6 +42,7 @@ _DAGNodeOperationType, DAGOperationGraphNode, _select_next_nodes, + _build_dag_node_operation_graph, ) from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy @@ -1202,87 +1203,6 @@ def _generate_dag_operation_graph_node( ) return actor_to_operation_nodes - def _build_dag_node_operation_graph( - self, - actor_to_operation_nodes: Dict[ - "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] - ], - ) -> Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]]: - """ - Generate a DAG node operation graph by adding edges based on the - following rules: - - #1 Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which - belong to the same task. - #2 Add an edge from COMPUTE with bind_index i to COMPUTE with bind_index - i+1 if they belong to the same actor. - #3 Add an edge from WRITE of the writer task to READ of the reader task. - - This is the step one of building an execution schedule for each actor. - - Args: - actor_to_operation_nodes: A dictionary that maps an actor handle to - a list of lists of DAGOperationGraphNode. For the same actor, the - index of the outer list corresponds to the index of the ExecutableTask - in the list of `executable_tasks` in `actor_to_executable_tasks`. In - the inner list, the order of operations is READ, COMPUTE, and WRITE. - - Returns: - A graph where each node is a DAGOperationGraphNode. The key is the index - of the task in idx_to_task, and the value is a dictionary that maps the - _DAGNodeOperationType (READ, COMPUTE, or WRITE) to the corresponding - DAGOperationGraphNode. - """ - assert self.idx_to_task - - graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]] = {} - - for _, operation_nodes_list in actor_to_operation_nodes.items(): - prev_compute_node = None - for operation_nodes in operation_nodes_list: - idx = operation_nodes[0].idx - read_node, compute_node, write_node = ( - operation_nodes[0], - operation_nodes[1], - operation_nodes[2], - ) - # Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which - # belong to the same task. - read_node.add_edge(compute_node) - compute_node.add_edge(write_node) - # Add an edge from COMPUTE with `bind_index` i to COMPUTE with - # `bind_index` i+1 if they belong to the same actor. - if prev_compute_node is not None: - prev_compute_node.add_edge(compute_node) - prev_compute_node = compute_node - assert idx not in graph - graph[idx] = { - _DAGNodeOperationType.READ: read_node, - _DAGNodeOperationType.COMPUTE: compute_node, - _DAGNodeOperationType.WRITE: write_node, - } - - from ray.dag import ( - ClassMethodNode, - MultiOutputNode, - ) - - # Add an edge from WRITE of the writer task to READ of the reader task. - for idx, task in self.idx_to_task.items(): - if not isinstance(task.dag_node, ClassMethodNode): - # The graph is used to generate an execution schedule for each actor. - # The edge from the InputNode has no impact on the final execution - # schedule. - continue - for downstream_idx in task.downstream_node_idxs: - downstream_dag_node = self.idx_to_task[downstream_idx].dag_node - if isinstance(downstream_dag_node, MultiOutputNode): - continue - graph[idx][_DAGNodeOperationType.WRITE].add_edge( - graph[downstream_idx][_DAGNodeOperationType.READ] - ) - return graph - def _build_execution_schedule(self): """ Generate an execution schedule for each actor. The schedule is a list of @@ -1318,7 +1238,9 @@ def _build_execution_schedule(self): # Step 1: Build a graph of DAGOperationGraphNode actor_to_operation_nodes = self._generate_dag_operation_graph_node() - graph = self._build_dag_node_operation_graph(actor_to_operation_nodes) + graph = _build_dag_node_operation_graph( + self.idx_to_task, actor_to_operation_nodes + ) # A dictionary mapping an actor id to a list of candidate nodes. The list # is maintained as a priority queue, so the head of the queue, i.e., diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 4df26a44e6ca5..fd3bf2389ca76 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -172,3 +172,86 @@ def _select_next_nodes( next_nodes.append(downstream_node) assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) return next_nodes + + +def _build_dag_node_operation_graph( + idx_to_task: Dict[int, "ray.dag.compiled_dag_node.CompiledTask"], + actor_to_operation_nodes: Dict[ + "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] + ], +) -> Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]]: + """ + Generate a DAG node operation graph by adding edges based on the + following rules: + + #1 Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which + belong to the same task. + #2 Add an edge from COMPUTE with bind_index i to COMPUTE with bind_index + i+1 if they belong to the same actor. + #3 Add an edge from WRITE of the writer task to READ of the reader task. + + This is the step one of building an execution schedule for each actor. + + Args: + idx_to_task: A dictionary that maps the `dag_index` to the `CompiledTask`. + `CompiledTask` contains information about a DAGNode and its downstream + nodes. + + actor_to_operation_nodes: A dictionary that maps an actor handle to + a list of lists of DAGOperationGraphNode. For the same actor, the + index of the outer list corresponds to the index of the ExecutableTask + in the list of `executable_tasks` in `actor_to_executable_tasks`. In + the inner list, the order of operations is READ, COMPUTE, and WRITE. + + Returns: + A graph where each node is a DAGOperationGraphNode. The key is the index + of the task in idx_to_task, and the value is a dictionary that maps the + _DAGNodeOperationType (READ, COMPUTE, or WRITE) to the corresponding + DAGOperationGraphNode. + """ + assert idx_to_task + graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]] = {} + + for _, operation_nodes_list in actor_to_operation_nodes.items(): + prev_compute_node = None + for operation_nodes in operation_nodes_list: + idx = operation_nodes[0].idx + read_node, compute_node, write_node = ( + operation_nodes[0], + operation_nodes[1], + operation_nodes[2], + ) + # Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which + # belong to the same task. + read_node.add_edge(compute_node) + compute_node.add_edge(write_node) + # Add an edge from COMPUTE with `bind_index` i to COMPUTE with + # `bind_index` i+1 if they belong to the same actor. + if prev_compute_node is not None: + prev_compute_node.add_edge(compute_node) + prev_compute_node = compute_node + assert idx not in graph + graph[idx] = { + _DAGNodeOperationType.READ: read_node, + _DAGNodeOperationType.COMPUTE: compute_node, + _DAGNodeOperationType.WRITE: write_node, + } + + # Import `ray.dag` here to avoid circular import. + from ray.dag import ClassMethodNode, MultiOutputNode + + # Add an edge from WRITE of the writer task to READ of the reader task. + for idx, task in idx_to_task.items(): + if not isinstance(task.dag_node, ClassMethodNode): + # The graph is used to generate an execution schedule for each actor. + # The edge from the InputNode has no impact on the final execution + # schedule. + continue + for downstream_idx in task.downstream_node_idxs: + downstream_dag_node = idx_to_task[downstream_idx].dag_node + if isinstance(downstream_dag_node, MultiOutputNode): + continue + graph[idx][_DAGNodeOperationType.WRITE].add_edge( + graph[downstream_idx][_DAGNodeOperationType.READ] + ) + return graph diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index a4a14e2f8f304..e5b40df6ea35d 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -14,8 +14,9 @@ DAGOperationGraphNode, DAGNodeOperation, _select_next_nodes, + _build_dag_node_operation_graph, ) -from ray.dag.compiled_dag_node import CompiledDAG, CompiledTask +from ray.dag.compiled_dag_node import CompiledTask import torch from typing import List, Dict, Tuple from dataclasses import dataclass, field @@ -811,8 +812,7 @@ def test_edges_between_read_compute_write(self, monkeypatch): monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) - compiled_dag = CompiledDAG() - compiled_dag.idx_to_task = { + idx_to_task = { 0: CompiledTask(0, InputNode()), 1: CompiledTask(1, ClassMethodNode()), 2: CompiledTask(2, MultiOutputNode()), @@ -827,7 +827,7 @@ def test_edges_between_read_compute_write(self, monkeypatch): ) ] } - graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + graph = _build_dag_node_operation_graph(idx_to_task, actor_to_operation_nodes) assert len(graph) == 1 self.check_edges_between_read_compute_write( @@ -846,14 +846,13 @@ def test_edge_between_writer_and_reader(self, monkeypatch): fake_actor_1, global_idx_1 = "fake_actor_1", 1 fake_actor_2, global_idx_2 = "fake_actor_2", 2 - compiled_dag = CompiledDAG() - compiled_dag.idx_to_task = { + idx_to_task = { 0: CompiledTask(0, InputNode()), 1: CompiledTask(1, ClassMethodNode()), 2: CompiledTask(2, ClassMethodNode()), 3: CompiledTask(3, MultiOutputNode()), } - compiled_dag.idx_to_task[1].downstream_node_idxs = {2: fake_actor_2} + idx_to_task[1].downstream_node_idxs = {2: fake_actor_2} actor_to_operation_nodes = { fake_actor_1: [ @@ -871,7 +870,7 @@ def test_edge_between_writer_and_reader(self, monkeypatch): ) ], } - graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + graph = _build_dag_node_operation_graph(idx_to_task, actor_to_operation_nodes) assert len(graph) == 2 self.check_edges_between_read_compute_write( @@ -895,16 +894,13 @@ def test_edge_between_compute_nodes(self, monkeypatch): fake_actor = "fake_actor" global_idx_1, global_idx_2 = 1, 2 - compiled_dag = CompiledDAG() - compiled_dag.idx_to_task = { + idx_to_task = { 0: CompiledTask(0, InputNode()), global_idx_1: CompiledTask(global_idx_1, ClassMethodNode()), global_idx_2: CompiledTask(global_idx_2, ClassMethodNode()), 3: CompiledTask(3, MultiOutputNode()), } - compiled_dag.idx_to_task[global_idx_1].downstream_node_idxs = { - global_idx_2: fake_actor - } + idx_to_task[global_idx_1].downstream_node_idxs = {global_idx_2: fake_actor} actor_to_operation_nodes = { fake_actor: [ @@ -920,7 +916,7 @@ def test_edge_between_compute_nodes(self, monkeypatch): ), ], } - graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + graph = _build_dag_node_operation_graph(idx_to_task, actor_to_operation_nodes) assert len(graph) == 2 self.check_edges_between_read_compute_write( @@ -948,8 +944,7 @@ def test_two_actors(self, monkeypatch): fake_actor_1, global_idx_1, global_idx_3 = "fake_actor_1", 1, 3 fake_actor_2, global_idx_2, global_idx_4 = "fake_actor_2", 2, 4 - compiled_dag = CompiledDAG() - compiled_dag.idx_to_task = { + idx_to_task = { 0: CompiledTask(0, InputNode()), global_idx_1: CompiledTask(global_idx_1, ClassMethodNode()), global_idx_2: CompiledTask(global_idx_2, ClassMethodNode()), @@ -957,12 +952,8 @@ def test_two_actors(self, monkeypatch): global_idx_4: CompiledTask(global_idx_4, ClassMethodNode()), 5: CompiledTask(5, MultiOutputNode()), } - compiled_dag.idx_to_task[global_idx_1].downstream_node_idxs = { - global_idx_4: fake_actor_2 - } - compiled_dag.idx_to_task[global_idx_2].downstream_node_idxs = { - global_idx_3: fake_actor_1 - } + idx_to_task[global_idx_1].downstream_node_idxs = {global_idx_4: fake_actor_2} + idx_to_task[global_idx_2].downstream_node_idxs = {global_idx_3: fake_actor_1} actor_to_operation_nodes = { fake_actor_1: [ @@ -990,7 +981,7 @@ def test_two_actors(self, monkeypatch): ), ], } - graph = compiled_dag._build_dag_node_operation_graph(actor_to_operation_nodes) + graph = _build_dag_node_operation_graph(idx_to_task, actor_to_operation_nodes) assert len(graph) == 4 self.check_edges_between_read_compute_write( From 164457e99ffaf9c13f3078096a6b5015d5eb78a8 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 16:24:47 +0000 Subject: [PATCH 085/111] remove print function --- python/ray/dag/compiled_dag_node.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 451b0662a1fbe..2a7563a4f3311 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1100,7 +1100,6 @@ def _get_or_compile( # Sort executable tasks based on their bind index, i.e., submission order # so that they will be executed in that order. executable_tasks.sort(key=lambda task: task.bind_index) - print(executable_tasks) self.actor_to_executable_tasks[actor_handle] = executable_tasks # Build an execution schedule for each actor From fadf05f14c17d756a156785765701f28939fc539 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 16:38:43 +0000 Subject: [PATCH 086/111] rename DAGOperationGraphNode to _DAGOperationGraphNode --- python/ray/dag/compiled_dag_node.py | 24 ++++++------ python/ray/dag/dag_node_operation.py | 37 +++++++++---------- .../experimental/test_execution_schedule.py | 14 +++---- 3 files changed, 37 insertions(+), 38 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 2a7563a4f3311..0c5ae5b577020 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -40,7 +40,7 @@ from ray.dag.dag_node_operation import ( DAGNodeOperation, _DAGNodeOperationType, - DAGOperationGraphNode, + _DAGOperationGraphNode, _select_next_nodes, _build_dag_node_operation_graph, ) @@ -1151,13 +1151,13 @@ def _get_or_compile( def _generate_dag_operation_graph_node( self, - ) -> Dict["ray.actor.ActorHandle", List[List[DAGOperationGraphNode]]]: + ) -> Dict["ray.actor.ActorHandle", List[List[_DAGOperationGraphNode]]]: """ Generate READ, COMPUTE, and WRITE operations for each DAG node. Returns: A dictionary that maps an actor handle to a list of lists of - DAGOperationGraphNode. For the same actor, the index of the + _DAGOperationGraphNode. For the same actor, the index of the outer list corresponds to the index of the ExecutableTask in the list of `executable_tasks` in `actor_to_executable_tasks`. In the inner list, the order of operations is READ, COMPUTE, @@ -1167,31 +1167,31 @@ def _generate_dag_operation_graph_node( assert self.actor_to_executable_tasks actor_to_operation_nodes: Dict[ - "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] + "ray.actor.ActorHandle", List[List[_DAGOperationGraphNode]] ] = defaultdict(list) for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): for local_idx, exec_task in enumerate(executable_tasks): - # Divide a DAG node into three DAGOperationGraphNodes: READ, COMPUTE, - # and WRITE. Each DAGOperationGraphNode has a DAGNodeOperation. + # Divide a DAG node into three _DAGOperationGraphNodes: READ, COMPUTE, + # and WRITE. Each _DAGOperationGraphNode has a DAGNodeOperation. idx = exec_task.idx dag_node = self.idx_to_task[idx].dag_node actor_handle = dag_node._get_actor_handle() requires_nccl = dag_node.type_hint.requires_nccl() - read_node = DAGOperationGraphNode( + read_node = _DAGOperationGraphNode( DAGNodeOperation(local_idx, _DAGNodeOperationType.READ), idx, actor_handle, requires_nccl, ) - compute_node = DAGOperationGraphNode( + compute_node = _DAGOperationGraphNode( DAGNodeOperation(local_idx, _DAGNodeOperationType.COMPUTE), idx, actor_handle, requires_nccl, ) - write_node = DAGOperationGraphNode( + write_node = _DAGOperationGraphNode( DAGNodeOperation(local_idx, _DAGNodeOperationType.WRITE), idx, actor_handle, @@ -1213,7 +1213,7 @@ def _build_execution_schedule(self): Step 2: Topological sort - It is possible to have multiple DAGOperationGraphNodes with zero in-degree. + It is possible to have multiple _DAGOperationGraphNodes with zero in-degree. Refer to the function `_select_next_nodes` for the logic of selecting nodes. Then, put the selected nodes into the corresponding actors' schedules. @@ -1235,7 +1235,7 @@ def _build_execution_schedule(self): "ray.actor.ActorHandle", List[DAGNodeOperation] ] = defaultdict(list) - # Step 1: Build a graph of DAGOperationGraphNode + # Step 1: Build a graph of _DAGOperationGraphNode actor_to_operation_nodes = self._generate_dag_operation_graph_node() graph = _build_dag_node_operation_graph( self.idx_to_task, actor_to_operation_nodes @@ -1245,7 +1245,7 @@ def _build_execution_schedule(self): # is maintained as a priority queue, so the head of the queue, i.e., # `candidates[0]`, is the node with the smallest `bind_index`. actor_to_candidates: Dict[ - "ray._raylet.ActorID", List[DAGOperationGraphNode] + "ray._raylet.ActorID", List[_DAGOperationGraphNode] ] = defaultdict(list) for _, node_dict in graph.items(): for _, node in node_dict.items(): diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index fd3bf2389ca76..ffff21ae2281b 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -38,9 +38,8 @@ def __init__( self.type = operation_type -@DeveloperAPI @total_ordering -class DAGOperationGraphNode: +class _DAGOperationGraphNode: def __init__( self, operation: DAGNodeOperation, @@ -49,7 +48,7 @@ def __init__( requires_nccl: bool, ): """ - DAGOperationGraphNode represents a node in the DAG operation graph. + _DAGOperationGraphNode represents a node in the DAG operation graph. It contains information about the node's in-degree, out-degree, edges, and the operation it performs. @@ -72,7 +71,7 @@ def in_degree(self) -> int: def __lt__(self, other): """ - Two DAGOperationGraphNodes are comparable only when they belong to + Two _DAGOperationGraphNodes are comparable only when they belong to the same actor. For operations on the same actor, if idx is smaller, the DAGNode to which this operation belongs has a smaller `bind_index`. """ @@ -81,7 +80,7 @@ def __lt__(self, other): def __eq__(self, other): """ - Two DAGOperationGraphNodes are comparable only when they belong to the + Two _DAGOperationGraphNodes are comparable only when they belong to the same actor. For operations on the same actor, two operations are equal only when they have the same `idx` and `type`. """ @@ -94,7 +93,7 @@ def __eq__(self, other): def __hash__(self): return hash((self.operation, self.idx)) - def add_edge(self, out_node: "DAGOperationGraphNode"): + def add_edge(self, out_node: "_DAGOperationGraphNode"): """ Add an edge from this node to `out_node`. An edge is a tuple of the operation's index and type. @@ -104,12 +103,12 @@ def add_edge(self, out_node: "DAGOperationGraphNode"): def _select_next_nodes( - actor_to_candidates: Dict["ray._raylet.ActorID", List[DAGOperationGraphNode]], - graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], + actor_to_candidates: Dict["ray._raylet.ActorID", List[_DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], ): """ This function selects the next nodes for topological sort to generate execution - schedule. If there are multiple candidate DAGOperationGraphNodes, select nodes + schedule. If there are multiple candidate _DAGOperationGraphNodes, select nodes based on the following rules: #1 If the nodes are not NCCL write nodes, select the one with the smallest @@ -133,13 +132,13 @@ def _select_next_nodes( the head of the queue, i.e., `candidates[0]`, is the node with the smallest `bind_index`. graph: A dictionary mapping the index of a task to a dictionary of its - DAGOperationGraphNodes for different operations. + _DAGOperationGraphNodes for different operations. Returns: - A list of DAGOperationGraphNodes to be placed into the corresponding + A list of _DAGOperationGraphNodes to be placed into the corresponding execution schedules. """ - next_nodes: List[DAGOperationGraphNode] = [] + next_nodes: List[_DAGOperationGraphNode] = [] for _, candidates in actor_to_candidates.items(): if not ( candidates[0].requires_nccl @@ -149,7 +148,7 @@ def _select_next_nodes( assert len(next_nodes) == 1 return next_nodes - first_nccl_node: Optional[DAGOperationGraphNode] = None + first_nccl_node: Optional[_DAGOperationGraphNode] = None for _, candidates in actor_to_candidates.items(): if ( candidates[0].requires_nccl @@ -177,9 +176,9 @@ def _select_next_nodes( def _build_dag_node_operation_graph( idx_to_task: Dict[int, "ray.dag.compiled_dag_node.CompiledTask"], actor_to_operation_nodes: Dict[ - "ray.actor.ActorHandle", List[List[DAGOperationGraphNode]] + "ray.actor.ActorHandle", List[List[_DAGOperationGraphNode]] ], -) -> Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]]: +) -> Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]]: """ Generate a DAG node operation graph by adding edges based on the following rules: @@ -198,19 +197,19 @@ def _build_dag_node_operation_graph( nodes. actor_to_operation_nodes: A dictionary that maps an actor handle to - a list of lists of DAGOperationGraphNode. For the same actor, the + a list of lists of _DAGOperationGraphNode. For the same actor, the index of the outer list corresponds to the index of the ExecutableTask in the list of `executable_tasks` in `actor_to_executable_tasks`. In the inner list, the order of operations is READ, COMPUTE, and WRITE. Returns: - A graph where each node is a DAGOperationGraphNode. The key is the index + A graph where each node is a _DAGOperationGraphNode. The key is the index of the task in idx_to_task, and the value is a dictionary that maps the _DAGNodeOperationType (READ, COMPUTE, or WRITE) to the corresponding - DAGOperationGraphNode. + _DAGOperationGraphNode. """ assert idx_to_task - graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]] = {} + graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]] = {} for _, operation_nodes_list in actor_to_operation_nodes.items(): prev_compute_node = None diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index e5b40df6ea35d..7b4d7784c21e1 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -11,7 +11,7 @@ from ray.dag import InputNode, MultiOutputNode, ClassMethodNode from ray.dag.dag_node_operation import ( _DAGNodeOperationType, - DAGOperationGraphNode, + _DAGOperationGraphNode, DAGNodeOperation, _select_next_nodes, _build_dag_node_operation_graph, @@ -533,7 +533,7 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): def generate_dag_graph_nodes(local_idx, global_idx, actor_handle, requires_nccl): graph_nodes = {} for op_type in _DAGNodeOperationType: - graph_nodes[op_type] = DAGOperationGraphNode( + graph_nodes[op_type] = _DAGOperationGraphNode( DAGNodeOperation(local_idx, op_type), global_idx, actor_handle, @@ -568,7 +568,7 @@ def test_two_candidates_on_same_actor(self): # The DAG node has a global index of 1, and its index in the # actor's `executable_tasks` list is 0. global_idx_1 = 1 - dag_node_1 = DAGOperationGraphNode( + dag_node_1 = _DAGOperationGraphNode( DAGNodeOperation(0, _DAGNodeOperationType.READ), global_idx_1, fake_actor, @@ -577,7 +577,7 @@ def test_two_candidates_on_same_actor(self): # The DAG node has a global index of 2, and its index in the # actor's `executable_tasks` list is 1. global_idx_2 = 2 - dag_node_2 = DAGOperationGraphNode( + dag_node_2 = _DAGOperationGraphNode( DAGNodeOperation(1, _DAGNodeOperationType.READ), global_idx_2, fake_actor, @@ -724,7 +724,7 @@ class TestBuildDAGNodeOperationGraph: def check_edges_between_read_compute_write( self, - graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], global_idx: int, expected_num_edges: List[Tuple[int, int]], ): @@ -757,7 +757,7 @@ def check_edges_between_read_compute_write( def check_edge_between_writer_and_reader( self, - graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], writer_global_idx: int, reader_global_idx: int, ): @@ -779,7 +779,7 @@ def check_edge_between_writer_and_reader( def check_edge_between_compute_nodes( self, - graph: Dict[int, Dict[_DAGNodeOperationType, DAGOperationGraphNode]], + graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], global_idx_1: int, global_idx_2: int, ): From 9895d8638cd53f53e6802b48ab0ea66c54ed12cb Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 16:44:27 +0000 Subject: [PATCH 087/111] rename DAGNodeOperation to _DAGNodeOperation --- python/ray/dag/compiled_dag_node.py | 22 +++++++++---------- python/ray/dag/dag_node_operation.py | 6 ++--- .../experimental/test_execution_schedule.py | 8 +++---- 3 files changed, 17 insertions(+), 19 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 0c5ae5b577020..c67ca8fa40465 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -38,7 +38,7 @@ ) from ray.dag.dag_node_operation import ( - DAGNodeOperation, + _DAGNodeOperation, _DAGNodeOperationType, _DAGOperationGraphNode, _select_next_nodes, @@ -93,16 +93,16 @@ def do_allocate_channel( def do_exec_tasks( self, tasks: List["ExecutableTask"], - schedule: List[DAGNodeOperation], + schedule: List[_DAGNodeOperation], ) -> None: """A generic actor method to begin executing the operations belonging to an - actor. This runs an infinite loop to execute each DAGNodeOperation in the + actor. This runs an infinite loop to execute each _DAGNodeOperation in the order specified by the schedule. It exits only if the actor dies or an exception is thrown. Args: tasks: the executable tasks corresponding to the actor methods. - schedule: A list of `DAGNodeOperations` that should be executed in order. + schedule: A list of _DAGNodeOperation that should be executed in order. """ try: for task in tasks: @@ -594,7 +594,7 @@ def __init__( # Mapping from the actor handle to the execution schedule which is a list # of operations to be executed. self.actor_to_execution_schedule: Dict[ - "ray.actor.ActorHandle", List[DAGNodeOperation] + "ray.actor.ActorHandle", List[_DAGNodeOperation] ] = defaultdict(list) # Mapping from the actor handle to the node ID that the actor is on. self.actor_to_node_id: Dict["ray.actor.ActorHandle", str] = {} @@ -1173,26 +1173,26 @@ def _generate_dag_operation_graph_node( for actor_handle, executable_tasks in self.actor_to_executable_tasks.items(): for local_idx, exec_task in enumerate(executable_tasks): # Divide a DAG node into three _DAGOperationGraphNodes: READ, COMPUTE, - # and WRITE. Each _DAGOperationGraphNode has a DAGNodeOperation. + # and WRITE. Each _DAGOperationGraphNode has a _DAGNodeOperation. idx = exec_task.idx dag_node = self.idx_to_task[idx].dag_node actor_handle = dag_node._get_actor_handle() requires_nccl = dag_node.type_hint.requires_nccl() read_node = _DAGOperationGraphNode( - DAGNodeOperation(local_idx, _DAGNodeOperationType.READ), + _DAGNodeOperation(local_idx, _DAGNodeOperationType.READ), idx, actor_handle, requires_nccl, ) compute_node = _DAGOperationGraphNode( - DAGNodeOperation(local_idx, _DAGNodeOperationType.COMPUTE), + _DAGNodeOperation(local_idx, _DAGNodeOperationType.COMPUTE), idx, actor_handle, requires_nccl, ) write_node = _DAGOperationGraphNode( - DAGNodeOperation(local_idx, _DAGNodeOperationType.WRITE), + _DAGNodeOperation(local_idx, _DAGNodeOperationType.WRITE), idx, actor_handle, requires_nccl, @@ -1205,7 +1205,7 @@ def _generate_dag_operation_graph_node( def _build_execution_schedule(self): """ Generate an execution schedule for each actor. The schedule is a list of - DAGNodeOperation. + _DAGNodeOperation. Step 1: Generate a DAG node operation graph. Refer to the functions `_generate_dag_operation_graph_node` and `_build_dag_node_operation_graph` @@ -1232,7 +1232,7 @@ def _build_execution_schedule(self): # Mapping from the actor handle to the execution schedule which is a list # of operations to be executed. actor_to_execution_schedule: Dict[ - "ray.actor.ActorHandle", List[DAGNodeOperation] + "ray.actor.ActorHandle", List[_DAGNodeOperation] ] = defaultdict(list) # Step 1: Build a graph of _DAGOperationGraphNode diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index ffff21ae2281b..c99407ef714fe 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -1,7 +1,6 @@ from functools import total_ordering from enum import Enum from typing import Set, Tuple, List, Dict, Optional -from ray.util.annotations import DeveloperAPI import ray import heapq @@ -19,8 +18,7 @@ class _DAGNodeOperationType(Enum): WRITE = "WRITE" -@DeveloperAPI -class DAGNodeOperation: +class _DAGNodeOperation: def __init__( self, idx: int, @@ -42,7 +40,7 @@ def __init__( class _DAGOperationGraphNode: def __init__( self, - operation: DAGNodeOperation, + operation: _DAGNodeOperation, idx: int, actor_handle: "ray.actor.ActorHandle", requires_nccl: bool, diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 7b4d7784c21e1..c3ff41ced5576 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -12,7 +12,7 @@ from ray.dag.dag_node_operation import ( _DAGNodeOperationType, _DAGOperationGraphNode, - DAGNodeOperation, + _DAGNodeOperation, _select_next_nodes, _build_dag_node_operation_graph, ) @@ -534,7 +534,7 @@ def generate_dag_graph_nodes(local_idx, global_idx, actor_handle, requires_nccl) graph_nodes = {} for op_type in _DAGNodeOperationType: graph_nodes[op_type] = _DAGOperationGraphNode( - DAGNodeOperation(local_idx, op_type), + _DAGNodeOperation(local_idx, op_type), global_idx, actor_handle, requires_nccl, @@ -569,7 +569,7 @@ def test_two_candidates_on_same_actor(self): # actor's `executable_tasks` list is 0. global_idx_1 = 1 dag_node_1 = _DAGOperationGraphNode( - DAGNodeOperation(0, _DAGNodeOperationType.READ), + _DAGNodeOperation(0, _DAGNodeOperationType.READ), global_idx_1, fake_actor, False, @@ -578,7 +578,7 @@ def test_two_candidates_on_same_actor(self): # actor's `executable_tasks` list is 1. global_idx_2 = 2 dag_node_2 = _DAGOperationGraphNode( - DAGNodeOperation(1, _DAGNodeOperationType.READ), + _DAGNodeOperation(1, _DAGNodeOperationType.READ), global_idx_2, fake_actor, False, From bcc10ad4fbc755153ca582e534d70511ccd655da Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 16:55:48 +0000 Subject: [PATCH 088/111] add comments for in_edges / out_edges --- python/ray/dag/dag_node_operation.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index c99407ef714fe..2b6d1a5462c37 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -60,6 +60,10 @@ def __init__( self.idx = idx self.actor_handle = actor_handle self.requires_nccl = requires_nccl + # The in_edges and out_edges are sets of tuples. Each tuple contains + # an integer `dag_idx`, which can be used to index into `idx_to_task` + # to get the corresponding task, and a `_DAGNodeOperationType`, which can + # be READ, COMPUTE, or WRITE. self.in_edges: Set[Tuple[int, _DAGNodeOperationType]] = set() self.out_edges: Set[Tuple[int, _DAGNodeOperationType]] = set() @@ -190,7 +194,7 @@ def _build_dag_node_operation_graph( This is the step one of building an execution schedule for each actor. Args: - idx_to_task: A dictionary that maps the `dag_index` to the `CompiledTask`. + idx_to_task: A dictionary that maps the `dag_idx` to the `CompiledTask`. `CompiledTask` contains information about a DAGNode and its downstream nodes. From b125f69b17aab36c8f91ecb0f27b995426ecc811 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 17:04:42 +0000 Subject: [PATCH 089/111] add comments for select_next_nodes behavior --- python/ray/dag/compiled_dag_node.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index c67ca8fa40465..5fd57dd588c43 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1261,6 +1261,10 @@ def _build_execution_schedule(self): # Step 2: Topological sort while actor_to_candidates: + # The function `_select_next_nodes` will pop a candidate node from + # `actor_to_candidates` and return a list of nodes that can be executed + # in the next step. If multiple nodes are returned, only the NCCL write + # node is popped in this iteration. nodes = _select_next_nodes(actor_to_candidates, graph) for node in nodes: if node in visited_nodes: From 45fb99fa9c2ebe3c80ad587888c1cb985ef9cc1c Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 17:30:13 +0000 Subject: [PATCH 090/111] add an example for actor_to_operation_nodes Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 5fd57dd588c43..a31a4bd7d289c 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1159,9 +1159,17 @@ def _generate_dag_operation_graph_node( A dictionary that maps an actor handle to a list of lists of _DAGOperationGraphNode. For the same actor, the index of the outer list corresponds to the index of the ExecutableTask in - the list of `executable_tasks` in `actor_to_executable_tasks`. - In the inner list, the order of operations is READ, COMPUTE, - and WRITE. + the list of `executable_tasks` in `actor_to_executable_tasks`, + i.e. `local_idx`. In the inner list, the order of operations + is READ, COMPUTE, and WRITE. + + Example: + { + actor1: [ + [READ COMPUTE WRITE] # local_idx 0 + [READ COMPUTE WRITE] # local_idx 1 + ] + } """ assert self.idx_to_task assert self.actor_to_executable_tasks From 72799f791151bb3774a424f11cdefad63d239026 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 20:53:03 +0000 Subject: [PATCH 091/111] split tests into cpu/gpu Signed-off-by: Kai-Hsun Chen --- python/ray/dag/BUILD | 5 +- .../experimental/test_execution_schedule.py | 501 ----------------- .../test_execution_schedule_gpu.py | 521 ++++++++++++++++++ 3 files changed, 524 insertions(+), 503 deletions(-) create mode 100644 python/ray/dag/tests/experimental/test_execution_schedule_gpu.py diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 1298d3ca1196b..9dd5f9aec1bb4 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -105,6 +105,7 @@ py_test_module_list( "tests/experimental/test_detect_deadlock_dag.py", "tests/experimental/test_multi_node_dag.py", "tests/experimental/test_torch_tensor_dag.py", + "tests/experimental/test_execution_schedule.py", ], tags = [ "accelerated_dag", @@ -151,10 +152,10 @@ py_test( name = "test_execution_schedule", size = "medium", srcs = [ - "tests/experimental/test_execution_schedule.py", + "tests/experimental/test_execution_schedule_gpu.py", ], env = {"RAY_PYTEST_USE_GPU": "1"}, - main = "tests/experimental/test_execution_schedule.py", + main = "tests/experimental/test_execution_schedule_gpu.py", tags = [ "accelerated_dag", "exclusive", diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index c3ff41ced5576..0b504b2a9eb00 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -4,9 +4,6 @@ import pytest -import ray -import ray.cluster_utils -from ray.experimental.channel.torch_tensor_type import TorchTensorType from ray.tests.conftest import * # noqa from ray.dag import InputNode, MultiOutputNode, ClassMethodNode from ray.dag.dag_node_operation import ( @@ -17,241 +14,12 @@ _build_dag_node_operation_graph, ) from ray.dag.compiled_dag_node import CompiledTask -import torch from typing import List, Dict, Tuple -from dataclasses import dataclass, field -from collections import deque, defaultdict from ray.actor import ActorHandle if sys.platform != "linux" and sys.platform != "darwin": pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) -USE_GPU = bool(os.environ.get("RAY_PYTEST_USE_GPU", 0)) - - -@dataclass -class PipelineConfig: - """ - pp_size: Number of pipeline parallel workers. - num_micro_batches: Number of micro-batches. - """ - - pp_size: int - num_micro_batches: int - - -@dataclass -class PipelineUnit: - """ - op: Operation type (FWD or BWD). - pp_rank: Pipeline parallel rank. - batch_id: Batch ID. - uid: Unique ID for the pipeline unit. - """ - - op: str - pp_rank: int - batch_id: int - uid: str = field(init=False, repr=False) - - def __post_init__(self): - self.uid = f"{self.op}_rank-{self.pp_rank}_batch-{self.batch_id}" - - def __repr__(self) -> str: - return self.uid - - -def generate_1f1b_schedule(config: PipelineConfig) -> List[List[PipelineUnit]]: - """ - Args: - config: Pipeline configuration. - Returns: - schedule: List of pipeline units for 1F1B pipeline parallelism. Each - inner list represents the schedule for a pipeline parallel worker. - """ - pp_size = config.pp_size - num_micro_batches = config.num_micro_batches - - schedule = [] - for pp_rank in range(config.pp_size): - warm_up_batches = pp_size - pp_rank - main_1f1b_batches = num_micro_batches - warm_up_batches - cool_down_batches = num_micro_batches - main_1f1b_batches - - rank_schedule = [] - bwd_batch_id = fwd_batch_id = 0 - - for _ in range(warm_up_batches): - rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) - fwd_batch_id += 1 - - for _ in range(main_1f1b_batches): - rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) - bwd_batch_id += 1 - rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) - fwd_batch_id += 1 - - for _ in range(cool_down_batches): - rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) - bwd_batch_id += 1 - schedule.append(rank_schedule) - return schedule - - -class PipelineModel: - def __init__( - self, - config: PipelineConfig, - schedule: List[List[PipelineUnit]], - blocks: List[ActorHandle], - ) -> None: - """ - Args: - config: Pipeline configuration. - schedule: List of pipeline units. Each inner list represents the - schedule for a pipeline parallel worker. - blocks: List of actors representing pipeline parallel workers. - """ - self.config = config - self.blocks = blocks - self.generate_pipeline_schedules(schedule) - self.dag = self.build_dag() - - def generate_pipeline_schedules(self, schedule: List[List[PipelineUnit]]): - """ - Convert per-worker schedule to per-batch schedule. - - Args: - schedule: List of pipeline units. Each inner list represents the - schedule for a pipeline parallel worker. - """ - self.id_to_unit = dict() - self.stage_schedules = defaultdict(list) - self.batch_schedules = defaultdict(list) - - for pp_rank, stage_schedule in enumerate(schedule): - self.stage_schedules[pp_rank] = stage_schedule - for unit in stage_schedule: - self.id_to_unit[unit.uid] = unit - self.batch_schedules[unit.batch_id].append(unit) - - for batch_id in self.batch_schedules: - fwd_units = [ - unit for unit in self.batch_schedules[batch_id] if unit.op == "FWD" - ] - bwd_units = [ - unit for unit in self.batch_schedules[batch_id] if unit.op == "BWD" - ] - - fwd_units.sort(key=lambda unit: unit.pp_rank) - bwd_units.sort(key=lambda unit: unit.pp_rank, reverse=True) - self.batch_schedules[batch_id] = fwd_units + bwd_units - - def build_dependency_graph(self): - """ - Add dependencies between pipeline units based on: - (1) Per-batch schedule and (2) Per-worker schedule. - """ - graph = defaultdict(set) - reversed_graph = defaultdict(set) - - for schedules in [self.batch_schedules, self.stage_schedules]: - for schedule in schedules.values(): - prev_unit = None - for unit in schedule: - if prev_unit: - graph[prev_unit.uid].add(unit.uid) - reversed_graph[unit.uid].add(prev_unit.uid) - prev_unit = unit - return graph, reversed_graph - - def build_dag(self): - """ - Build accelerated DAG for the pipeline model. - """ - graph, reversed_graph = self.build_dependency_graph() - dag_nodes = dict() # Cache DAG Node for each unit - - first_unit = self.batch_schedules[0][0] - queue = deque([first_unit.uid]) - - with InputNode() as input_node: - root_node = self.blocks[0].read_input.bind(input_node) - - output_nodes = [] - - while queue: - uid = queue.popleft() - unit = self.id_to_unit[uid] - batch_id = unit.batch_id - batch_schedule_index = self.batch_schedules[batch_id].index(unit) - - # First forward step - if batch_schedule_index == 0: - prev_dag_node = root_node - else: - prev_unit = self.batch_schedules[batch_id][batch_schedule_index - 1] - prev_dag_node = dag_nodes[prev_unit.uid] - - block = self.blocks[unit.pp_rank] - if unit.op == "FWD": - cur_dag_node = block.fwd.bind(prev_dag_node) - else: - cur_dag_node = block.bwd.bind(prev_dag_node) - - # Last backward step - if batch_schedule_index == 2 * self.config.pp_size - 1: - output_nodes.append(cur_dag_node) - - # ADD NCCL Channel: - if unit.op == "FWD" and unit.pp_rank < self.config.pp_size - 1: - cur_dag_node.with_type_hint( - TorchTensorType(transport=TorchTensorType.NCCL) - ) - if unit.op == "BWD" and unit.pp_rank > 0: - cur_dag_node.with_type_hint( - TorchTensorType(transport=TorchTensorType.NCCL) - ) - - dag_nodes[uid] = cur_dag_node - - # Enqueue new units - for target_uid in graph[uid]: - reversed_graph[target_uid].remove(uid) - if not reversed_graph[target_uid]: - queue.append(target_uid) - - dag = MultiOutputNode(output_nodes) - compiled_dag = dag.experimental_compile() - return compiled_dag - - def step(self, input_batches): - return ray.get(self.dag.execute(input_batches)) - - def teardown(self): - self.dag.teardown() - - -@ray.remote(num_cpus=0, num_gpus=1) -class Worker: - def __init__(self): - pass - - def fwd(self, value): - return value - - def bwd(self, value): - return value - - def read_input(self, input): - return input - - def no_op(self, value): - return value - - def no_op_two(self, value1, value2): - return value1, value2 - def mock_actor_handle_init(self, actor_id: str): self._ray_actor_id = actor_id @@ -261,275 +29,6 @@ def mock_init(self): pass -@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) -def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): - """ - This test simulates a simple 1F1B pipeline parallelism for training with - 2 workers and 2 batches. - - w1: fwd_b1 fwd_b2 bwd_b1 bwd_b2 - w2: fwd_b1 bwd_b1 fwd_b2 bwd_b2 - - The communication between workers is done using NCCL. The communication - within the worker actor is done using IntraProcessChannel. - """ - if not USE_GPU: - pytest.skip("NCCL tests require GPUs") - - monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) - - w1 = Worker.remote() - w2 = Worker.remote() - - with InputNode() as inp: - w1_input = w1.read_input.bind(inp) - batch_1 = w1.fwd.bind(w1_input) - batch_1.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) - batch_2 = w1.fwd.bind(w1_input) - batch_2.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) - batch_1 = w2.fwd.bind(batch_1) - batch_1 = w2.bwd.bind(batch_1) - batch_1.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) - batch_2 = w2.fwd.bind(batch_2) - batch_1 = w1.bwd.bind(batch_1) - batch_2 = w2.bwd.bind(batch_2) - batch_2.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) - batch_2 = w1.bwd.bind(batch_2) - dag = MultiOutputNode( - [ - batch_1, - batch_2, - ] - ) - compiled_dag = dag.experimental_compile() - - w1_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (0, _DAGNodeOperationType.WRITE), - (1, _DAGNodeOperationType.READ), - (1, _DAGNodeOperationType.COMPUTE), - (1, _DAGNodeOperationType.WRITE), - (2, _DAGNodeOperationType.READ), - (2, _DAGNodeOperationType.COMPUTE), - (2, _DAGNodeOperationType.WRITE), - (3, _DAGNodeOperationType.READ), - (3, _DAGNodeOperationType.COMPUTE), - (3, _DAGNodeOperationType.WRITE), - (4, _DAGNodeOperationType.READ), - (4, _DAGNodeOperationType.COMPUTE), - (4, _DAGNodeOperationType.WRITE), - ] - w2_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (0, _DAGNodeOperationType.WRITE), - (1, _DAGNodeOperationType.READ), - (1, _DAGNodeOperationType.COMPUTE), - (2, _DAGNodeOperationType.READ), - (1, _DAGNodeOperationType.WRITE), - (2, _DAGNodeOperationType.COMPUTE), - (2, _DAGNodeOperationType.WRITE), - (3, _DAGNodeOperationType.READ), - (3, _DAGNodeOperationType.COMPUTE), - (3, _DAGNodeOperationType.WRITE), - ] - w1_schedule = compiled_dag.actor_to_execution_schedule[w1] - w2_schedule = compiled_dag.actor_to_execution_schedule[w2] - - for schedule, expected_schedule in zip( - [w1_schedule, w2_schedule], [w1_expected_schedule, w2_expected_schedule] - ): - assert len(schedule) == len(expected_schedule) - for i, operation in enumerate(schedule): - assert operation.idx == expected_schedule[i][0] - assert operation.type == expected_schedule[i][1] - - tensor_cpu = torch.zeros(10, 10) - ref = compiled_dag.execute(tensor_cpu) - tensors = ray.get(ref) - tensor_cuda = tensor_cpu.to("cuda:0") - - assert len(tensors) == 2 - for t in tensors: - assert torch.equal(t, tensor_cuda) - - compiled_dag.teardown() - - -@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 4}], indirect=True) -def test_simulate_pp_4workers_8batches_1f1b(ray_start_regular, monkeypatch): - """ - This test simulates a 1F1B pipeline parallelism for training with - 4 workers and 8 batches. - """ - if not USE_GPU: - pytest.skip("NCCL tests require GPUs") - - monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) - - num_worker, num_batch = 4, 8 - - workers = [Worker.remote() for _ in range(num_worker)] - config = PipelineConfig(num_worker, num_batch) - schedule = generate_1f1b_schedule(config) - model = PipelineModel(config, schedule, workers) - - tensor_cpu = torch.zeros(10, 10) - tensors = model.step(tensor_cpu) - tensor_cuda = tensor_cpu.to("cuda:0") - assert len(tensors) == num_batch - for t in tensors: - assert torch.equal(t, tensor_cuda) - model.teardown() - - -@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) -def test_three_actors_with_nccl_1(ray_start_regular): - """ - Driver -> a.no_op -> b.no_op -> a.no_op_two -> Driver - | | - -> c.no_op - - """ - if not USE_GPU: - pytest.skip("NCCL tests require GPUs") - - a = Worker.remote() - b = Worker.remote() - c = Worker.remote() - - with InputNode() as inp: - dag = a.no_op.bind(inp) - dag.with_type_hint(TorchTensorType(transport="nccl")) - branch1 = b.no_op.bind(dag) - branch1.with_type_hint(TorchTensorType(transport="nccl")) - branch2 = c.no_op.bind(dag) - branch2.with_type_hint(TorchTensorType(transport="nccl")) - dag = a.no_op_two.bind(branch1, branch2) - - compiled_dag = dag.experimental_compile() - - a_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (0, _DAGNodeOperationType.WRITE), - (1, _DAGNodeOperationType.READ), - (1, _DAGNodeOperationType.COMPUTE), - (1, _DAGNodeOperationType.WRITE), - ] - b_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (0, _DAGNodeOperationType.WRITE), - ] - c_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (0, _DAGNodeOperationType.WRITE), - ] - a_schedule = compiled_dag.actor_to_execution_schedule[a] - b_schedule = compiled_dag.actor_to_execution_schedule[b] - c_schedule = compiled_dag.actor_to_execution_schedule[c] - - for schedule, expected_schedule in zip( - [a_schedule, b_schedule, c_schedule], - [a_expected_schedule, b_expected_schedule, c_expected_schedule], - ): - assert len(schedule) == len(expected_schedule) - for i, operation in enumerate(schedule): - assert operation.idx == expected_schedule[i][0] - assert operation.type == expected_schedule[i][1] - - tensor_cpu = torch.zeros(10, 10) - ref = compiled_dag.execute(tensor_cpu) - tensors = ray.get(ref) - tensor_cuda = tensor_cpu.to("cuda:0") - - assert len(tensors) == 2 - for t in tensors: - assert torch.equal(t, tensor_cuda) - - compiled_dag.teardown() - - -@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) -def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): - if not USE_GPU: - pytest.skip("NCCL tests require GPUs") - - monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) - - a = Worker.remote() - b = Worker.remote() - c = Worker.remote() - - with InputNode() as inp: - branch1 = a.no_op.bind(inp) - branch1.with_type_hint(TorchTensorType(transport="nccl")) - branch2 = b.no_op.bind(inp) - branch2.with_type_hint(TorchTensorType(transport="nccl")) - branch3 = c.no_op.bind(inp) - branch3.with_type_hint(TorchTensorType(transport="nccl")) - dag = MultiOutputNode( - [ - a.no_op.bind(branch3), - b.no_op.bind(branch1), - c.no_op.bind(branch2), - ] - ) - - compiled_dag = dag.experimental_compile() - - a_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (1, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.WRITE), - (1, _DAGNodeOperationType.COMPUTE), - (1, _DAGNodeOperationType.WRITE), - ] - b_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (1, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.WRITE), - (1, _DAGNodeOperationType.COMPUTE), - (1, _DAGNodeOperationType.WRITE), - ] - c_expected_schedule = [ - (0, _DAGNodeOperationType.READ), - (0, _DAGNodeOperationType.COMPUTE), - (0, _DAGNodeOperationType.WRITE), - (1, _DAGNodeOperationType.READ), - (1, _DAGNodeOperationType.COMPUTE), - (1, _DAGNodeOperationType.WRITE), - ] - - a_schedule = compiled_dag.actor_to_execution_schedule[a] - b_schedule = compiled_dag.actor_to_execution_schedule[b] - c_schedule = compiled_dag.actor_to_execution_schedule[c] - - for schedule, expected_schedule in zip( - [a_schedule, b_schedule, c_schedule], - [a_expected_schedule, b_expected_schedule, c_expected_schedule], - ): - assert len(schedule) == len(expected_schedule) - for i, operation in enumerate(schedule): - assert operation.idx == expected_schedule[i][0] - assert operation.type == expected_schedule[i][1] - - tensor_cpu = torch.zeros(10, 10) - ref = compiled_dag.execute(tensor_cpu) - tensors = ray.get(ref) - tensor_cuda = tensor_cpu.to("cuda:0") - - assert len(tensors) == 3 - for t in tensors: - assert torch.equal(t, tensor_cuda) - - compiled_dag.teardown() - - def generate_dag_graph_nodes(local_idx, global_idx, actor_handle, requires_nccl): graph_nodes = {} for op_type in _DAGNodeOperationType: diff --git a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py new file mode 100644 index 0000000000000..a9408d4f8db38 --- /dev/null +++ b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py @@ -0,0 +1,521 @@ +# coding: utf-8 +import os +import sys + +import pytest + +import ray +import ray.cluster_utils +from ray.experimental.channel.torch_tensor_type import TorchTensorType +from ray.tests.conftest import * # noqa +from ray.dag import InputNode, MultiOutputNode +from ray.dag.dag_node_operation import _DAGNodeOperationType +import torch +from typing import List +from dataclasses import dataclass, field +from collections import deque, defaultdict +from ray.actor import ActorHandle + +if sys.platform != "linux" and sys.platform != "darwin": + pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) + +USE_GPU = bool(os.environ.get("RAY_PYTEST_USE_GPU", 0)) + + +@dataclass +class PipelineConfig: + """ + pp_size: Number of pipeline parallel workers. + num_micro_batches: Number of micro-batches. + """ + + pp_size: int + num_micro_batches: int + + +@dataclass +class PipelineUnit: + """ + op: Operation type (FWD or BWD). + pp_rank: Pipeline parallel rank. + batch_id: Batch ID. + uid: Unique ID for the pipeline unit. + """ + + op: str + pp_rank: int + batch_id: int + uid: str = field(init=False, repr=False) + + def __post_init__(self): + self.uid = f"{self.op}_rank-{self.pp_rank}_batch-{self.batch_id}" + + def __repr__(self) -> str: + return self.uid + + +def generate_1f1b_schedule(config: PipelineConfig) -> List[List[PipelineUnit]]: + """ + Args: + config: Pipeline configuration. + Returns: + schedule: List of pipeline units for 1F1B pipeline parallelism. Each + inner list represents the schedule for a pipeline parallel worker. + """ + pp_size = config.pp_size + num_micro_batches = config.num_micro_batches + + schedule = [] + for pp_rank in range(config.pp_size): + warm_up_batches = pp_size - pp_rank + main_1f1b_batches = num_micro_batches - warm_up_batches + cool_down_batches = num_micro_batches - main_1f1b_batches + + rank_schedule = [] + bwd_batch_id = fwd_batch_id = 0 + + for _ in range(warm_up_batches): + rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) + fwd_batch_id += 1 + + for _ in range(main_1f1b_batches): + rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) + bwd_batch_id += 1 + rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) + fwd_batch_id += 1 + + for _ in range(cool_down_batches): + rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) + bwd_batch_id += 1 + schedule.append(rank_schedule) + return schedule + + +class PipelineModel: + def __init__( + self, + config: PipelineConfig, + schedule: List[List[PipelineUnit]], + blocks: List[ActorHandle], + ) -> None: + """ + Args: + config: Pipeline configuration. + schedule: List of pipeline units. Each inner list represents the + schedule for a pipeline parallel worker. + blocks: List of actors representing pipeline parallel workers. + """ + self.config = config + self.blocks = blocks + self.generate_pipeline_schedules(schedule) + self.dag = self.build_dag() + + def generate_pipeline_schedules(self, schedule: List[List[PipelineUnit]]): + """ + Convert per-worker schedule to per-batch schedule. + + Args: + schedule: List of pipeline units. Each inner list represents the + schedule for a pipeline parallel worker. + """ + self.id_to_unit = dict() + self.stage_schedules = defaultdict(list) + self.batch_schedules = defaultdict(list) + + for pp_rank, stage_schedule in enumerate(schedule): + self.stage_schedules[pp_rank] = stage_schedule + for unit in stage_schedule: + self.id_to_unit[unit.uid] = unit + self.batch_schedules[unit.batch_id].append(unit) + + for batch_id in self.batch_schedules: + fwd_units = [ + unit for unit in self.batch_schedules[batch_id] if unit.op == "FWD" + ] + bwd_units = [ + unit for unit in self.batch_schedules[batch_id] if unit.op == "BWD" + ] + + fwd_units.sort(key=lambda unit: unit.pp_rank) + bwd_units.sort(key=lambda unit: unit.pp_rank, reverse=True) + self.batch_schedules[batch_id] = fwd_units + bwd_units + + def build_dependency_graph(self): + """ + Add dependencies between pipeline units based on: + (1) Per-batch schedule and (2) Per-worker schedule. + """ + graph = defaultdict(set) + reversed_graph = defaultdict(set) + + for schedules in [self.batch_schedules, self.stage_schedules]: + for schedule in schedules.values(): + prev_unit = None + for unit in schedule: + if prev_unit: + graph[prev_unit.uid].add(unit.uid) + reversed_graph[unit.uid].add(prev_unit.uid) + prev_unit = unit + return graph, reversed_graph + + def build_dag(self): + """ + Build accelerated DAG for the pipeline model. + """ + graph, reversed_graph = self.build_dependency_graph() + dag_nodes = dict() # Cache DAG Node for each unit + + first_unit = self.batch_schedules[0][0] + queue = deque([first_unit.uid]) + + with InputNode() as input_node: + root_node = self.blocks[0].read_input.bind(input_node) + + output_nodes = [] + + while queue: + uid = queue.popleft() + unit = self.id_to_unit[uid] + batch_id = unit.batch_id + batch_schedule_index = self.batch_schedules[batch_id].index(unit) + + # First forward step + if batch_schedule_index == 0: + prev_dag_node = root_node + else: + prev_unit = self.batch_schedules[batch_id][batch_schedule_index - 1] + prev_dag_node = dag_nodes[prev_unit.uid] + + block = self.blocks[unit.pp_rank] + if unit.op == "FWD": + cur_dag_node = block.fwd.bind(prev_dag_node) + else: + cur_dag_node = block.bwd.bind(prev_dag_node) + + # Last backward step + if batch_schedule_index == 2 * self.config.pp_size - 1: + output_nodes.append(cur_dag_node) + + # ADD NCCL Channel: + if unit.op == "FWD" and unit.pp_rank < self.config.pp_size - 1: + cur_dag_node.with_type_hint( + TorchTensorType(transport=TorchTensorType.NCCL) + ) + if unit.op == "BWD" and unit.pp_rank > 0: + cur_dag_node.with_type_hint( + TorchTensorType(transport=TorchTensorType.NCCL) + ) + + dag_nodes[uid] = cur_dag_node + + # Enqueue new units + for target_uid in graph[uid]: + reversed_graph[target_uid].remove(uid) + if not reversed_graph[target_uid]: + queue.append(target_uid) + + dag = MultiOutputNode(output_nodes) + compiled_dag = dag.experimental_compile() + return compiled_dag + + def step(self, input_batches): + return ray.get(self.dag.execute(input_batches)) + + def teardown(self): + self.dag.teardown() + + +@ray.remote(num_cpus=0, num_gpus=1) +class Worker: + def __init__(self): + pass + + def fwd(self, value): + return value + + def bwd(self, value): + return value + + def read_input(self, input): + return input + + def no_op(self, value): + return value + + def no_op_two(self, value1, value2): + return value1, value2 + + +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) +def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): + """ + This test simulates a simple 1F1B pipeline parallelism for training with + 2 workers and 2 batches. + + w1: fwd_b1 fwd_b2 bwd_b1 bwd_b2 + w2: fwd_b1 bwd_b1 fwd_b2 bwd_b2 + + The communication between workers is done using NCCL. The communication + within the worker actor is done using IntraProcessChannel. + """ + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) + + w1 = Worker.remote() + w2 = Worker.remote() + + with InputNode() as inp: + w1_input = w1.read_input.bind(inp) + batch_1 = w1.fwd.bind(w1_input) + batch_1.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_2 = w1.fwd.bind(w1_input) + batch_2.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_1 = w2.fwd.bind(batch_1) + batch_1 = w2.bwd.bind(batch_1) + batch_1.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_2 = w2.fwd.bind(batch_2) + batch_1 = w1.bwd.bind(batch_1) + batch_2 = w2.bwd.bind(batch_2) + batch_2.with_type_hint(TorchTensorType(transport=TorchTensorType.NCCL)) + batch_2 = w1.bwd.bind(batch_2) + dag = MultiOutputNode( + [ + batch_1, + batch_2, + ] + ) + compiled_dag = dag.experimental_compile() + + w1_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), + (2, _DAGNodeOperationType.READ), + (2, _DAGNodeOperationType.COMPUTE), + (2, _DAGNodeOperationType.WRITE), + (3, _DAGNodeOperationType.READ), + (3, _DAGNodeOperationType.COMPUTE), + (3, _DAGNodeOperationType.WRITE), + (4, _DAGNodeOperationType.READ), + (4, _DAGNodeOperationType.COMPUTE), + (4, _DAGNodeOperationType.WRITE), + ] + w2_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (2, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.WRITE), + (2, _DAGNodeOperationType.COMPUTE), + (2, _DAGNodeOperationType.WRITE), + (3, _DAGNodeOperationType.READ), + (3, _DAGNodeOperationType.COMPUTE), + (3, _DAGNodeOperationType.WRITE), + ] + w1_schedule = compiled_dag.actor_to_execution_schedule[w1] + w2_schedule = compiled_dag.actor_to_execution_schedule[w2] + + for schedule, expected_schedule in zip( + [w1_schedule, w2_schedule], [w1_expected_schedule, w2_expected_schedule] + ): + assert len(schedule) == len(expected_schedule) + for i, operation in enumerate(schedule): + assert operation.idx == expected_schedule[i][0] + assert operation.type == expected_schedule[i][1] + + tensor_cpu = torch.zeros(10, 10) + ref = compiled_dag.execute(tensor_cpu) + tensors = ray.get(ref) + tensor_cuda = tensor_cpu.to("cuda:0") + + assert len(tensors) == 2 + for t in tensors: + assert torch.equal(t, tensor_cuda) + + compiled_dag.teardown() + + +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 4}], indirect=True) +def test_simulate_pp_4workers_8batches_1f1b(ray_start_regular, monkeypatch): + """ + This test simulates a 1F1B pipeline parallelism for training with + 4 workers and 8 batches. + """ + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) + + num_worker, num_batch = 4, 8 + + workers = [Worker.remote() for _ in range(num_worker)] + config = PipelineConfig(num_worker, num_batch) + schedule = generate_1f1b_schedule(config) + model = PipelineModel(config, schedule, workers) + + tensor_cpu = torch.zeros(10, 10) + tensors = model.step(tensor_cpu) + tensor_cuda = tensor_cpu.to("cuda:0") + assert len(tensors) == num_batch + for t in tensors: + assert torch.equal(t, tensor_cuda) + model.teardown() + + +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) +def test_three_actors_with_nccl_1(ray_start_regular): + """ + Driver -> a.no_op -> b.no_op -> a.no_op_two -> Driver + | | + -> c.no_op - + """ + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + a = Worker.remote() + b = Worker.remote() + c = Worker.remote() + + with InputNode() as inp: + dag = a.no_op.bind(inp) + dag.with_type_hint(TorchTensorType(transport="nccl")) + branch1 = b.no_op.bind(dag) + branch1.with_type_hint(TorchTensorType(transport="nccl")) + branch2 = c.no_op.bind(dag) + branch2.with_type_hint(TorchTensorType(transport="nccl")) + dag = a.no_op_two.bind(branch1, branch2) + + compiled_dag = dag.experimental_compile() + + a_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), + ] + b_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + ] + c_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + ] + a_schedule = compiled_dag.actor_to_execution_schedule[a] + b_schedule = compiled_dag.actor_to_execution_schedule[b] + c_schedule = compiled_dag.actor_to_execution_schedule[c] + + for schedule, expected_schedule in zip( + [a_schedule, b_schedule, c_schedule], + [a_expected_schedule, b_expected_schedule, c_expected_schedule], + ): + assert len(schedule) == len(expected_schedule) + for i, operation in enumerate(schedule): + assert operation.idx == expected_schedule[i][0] + assert operation.type == expected_schedule[i][1] + + tensor_cpu = torch.zeros(10, 10) + ref = compiled_dag.execute(tensor_cpu) + tensors = ray.get(ref) + tensor_cuda = tensor_cpu.to("cuda:0") + + assert len(tensors) == 2 + for t in tensors: + assert torch.equal(t, tensor_cuda) + + compiled_dag.teardown() + + +@pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) +def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) + + a = Worker.remote() + b = Worker.remote() + c = Worker.remote() + + with InputNode() as inp: + branch1 = a.no_op.bind(inp) + branch1.with_type_hint(TorchTensorType(transport="nccl")) + branch2 = b.no_op.bind(inp) + branch2.with_type_hint(TorchTensorType(transport="nccl")) + branch3 = c.no_op.bind(inp) + branch3.with_type_hint(TorchTensorType(transport="nccl")) + dag = MultiOutputNode( + [ + a.no_op.bind(branch3), + b.no_op.bind(branch1), + c.no_op.bind(branch2), + ] + ) + + compiled_dag = dag.experimental_compile() + + a_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), + ] + b_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), + ] + c_expected_schedule = [ + (0, _DAGNodeOperationType.READ), + (0, _DAGNodeOperationType.COMPUTE), + (0, _DAGNodeOperationType.WRITE), + (1, _DAGNodeOperationType.READ), + (1, _DAGNodeOperationType.COMPUTE), + (1, _DAGNodeOperationType.WRITE), + ] + + a_schedule = compiled_dag.actor_to_execution_schedule[a] + b_schedule = compiled_dag.actor_to_execution_schedule[b] + c_schedule = compiled_dag.actor_to_execution_schedule[c] + + for schedule, expected_schedule in zip( + [a_schedule, b_schedule, c_schedule], + [a_expected_schedule, b_expected_schedule, c_expected_schedule], + ): + assert len(schedule) == len(expected_schedule) + for i, operation in enumerate(schedule): + assert operation.idx == expected_schedule[i][0] + assert operation.type == expected_schedule[i][1] + + tensor_cpu = torch.zeros(10, 10) + ref = compiled_dag.execute(tensor_cpu) + tensors = ray.get(ref) + tensor_cuda = tensor_cpu.to("cuda:0") + + assert len(tensors) == 3 + for t in tensors: + assert torch.equal(t, tensor_cuda) + + compiled_dag.teardown() + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) From 67857a56e655f779aa7470d1d9920614d4a1f740 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 21:09:29 +0000 Subject: [PATCH 092/111] add comments for ExecutableTask Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 18 ++++++++++++++++++ .../test_execution_schedule_gpu.py | 1 + 2 files changed, 19 insertions(+) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index a31a4bd7d289c..7c3b4f8750230 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -337,6 +337,11 @@ def __init__( self._intermediate_buffer: Any = None def cancel(self): + """ + Close all the input channels and the output channel. The exact behavior + depends on the type of channel. Typically, it will release the resources + used by the channels. + """ self.input_reader.close() self.output_writer.close() @@ -352,10 +357,23 @@ def prepare(self): self.output_writer.start() def set_intermediate_buffer(self, data: Any): + """ + Store the intermediate result of a READ or COMPUTE operation. + + Args: + data: The intermediate result of a READ or COMPUTE operation. + """ assert self._intermediate_buffer is None self._intermediate_buffer = data def reset_intermediate_buffer(self) -> Any: + """ + Retrieve the intermediate result of a READ or COMPUTE operation, + and reset the intermediate buffer to None. + + Returns: + The intermediate result of a READ or COMPUTE operation. + """ data = self._intermediate_buffer self._intermediate_buffer = None return data diff --git a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py index a9408d4f8db38..e1a123d980d11 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py @@ -514,6 +514,7 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): compiled_dag.teardown() + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From e0df08bf80446f0f238a1926ac3add7d58916d1d Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 21:20:53 +0000 Subject: [PATCH 093/111] move add_edge into a top-level function Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 26 +++++----- .../experimental/test_execution_schedule.py | 47 ++++++++++++------- 2 files changed, 43 insertions(+), 30 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 2b6d1a5462c37..4ed10f4ae6f82 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -95,13 +95,14 @@ def __eq__(self, other): def __hash__(self): return hash((self.operation, self.idx)) - def add_edge(self, out_node: "_DAGOperationGraphNode"): - """ - Add an edge from this node to `out_node`. An edge is a tuple of - the operation's index and type. - """ - self.out_edges.add((out_node.idx, out_node.operation.type)) - out_node.in_edges.add((self.idx, self.operation.type)) + +def _add_edge(from_node: _DAGOperationGraphNode, to_node: _DAGOperationGraphNode): + """ + Add an edge from `from_node` to `to_node`. An edge is a tuple of + the operation's index and type. + """ + from_node.out_edges.add((to_node.idx, to_node.operation.type)) + to_node.in_edges.add((from_node.idx, from_node.operation.type)) def _select_next_nodes( @@ -224,12 +225,12 @@ def _build_dag_node_operation_graph( ) # Add edges from READ to COMPUTE, and from COMPUTE to WRITE, which # belong to the same task. - read_node.add_edge(compute_node) - compute_node.add_edge(write_node) + _add_edge(read_node, compute_node) + _add_edge(compute_node, write_node) # Add an edge from COMPUTE with `bind_index` i to COMPUTE with # `bind_index` i+1 if they belong to the same actor. if prev_compute_node is not None: - prev_compute_node.add_edge(compute_node) + _add_edge(prev_compute_node, compute_node) prev_compute_node = compute_node assert idx not in graph graph[idx] = { @@ -252,7 +253,8 @@ def _build_dag_node_operation_graph( downstream_dag_node = idx_to_task[downstream_idx].dag_node if isinstance(downstream_dag_node, MultiOutputNode): continue - graph[idx][_DAGNodeOperationType.WRITE].add_edge( - graph[downstream_idx][_DAGNodeOperationType.READ] + _add_edge( + graph[idx][_DAGNodeOperationType.WRITE], + graph[downstream_idx][_DAGNodeOperationType.READ], ) return graph diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 0b504b2a9eb00..feae04a2f4539 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -12,6 +12,7 @@ _DAGNodeOperation, _select_next_nodes, _build_dag_node_operation_graph, + _add_edge, ) from ray.dag.compiled_dag_node import CompiledTask from typing import List, Dict, Tuple @@ -118,14 +119,18 @@ def test_only_one_nccl_write(self, monkeypatch): } del mock_graph[global_idx_1][_DAGNodeOperationType.READ] del mock_graph[global_idx_1][_DAGNodeOperationType.COMPUTE] - mock_graph[global_idx_1][_DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_2][_DAGNodeOperationType.READ] + + _add_edge( + mock_graph[global_idx_1][_DAGNodeOperationType.WRITE], + mock_graph[global_idx_2][_DAGNodeOperationType.READ], ) - mock_graph[global_idx_2][_DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE] + _add_edge( + mock_graph[global_idx_2][_DAGNodeOperationType.READ], + mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE], ) - mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_2][_DAGNodeOperationType.WRITE] + _add_edge( + mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE], + mock_graph[global_idx_2][_DAGNodeOperationType.WRITE], ) mock_actor_to_candidates = { fake_actor_1: [mock_graph[global_idx_1][_DAGNodeOperationType.WRITE]], @@ -181,23 +186,29 @@ def test_two_nccl_writes(self, monkeypatch): del mock_graph[global_idx_2_0][_DAGNodeOperationType.READ] del mock_graph[global_idx_2_0][_DAGNodeOperationType.COMPUTE] - mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] + _add_edge( + mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE], + mock_graph[global_idx_2_1][_DAGNodeOperationType.READ], ) - mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE].add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.READ] + _add_edge( + mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE], + mock_graph[global_idx_1_1][_DAGNodeOperationType.READ], ) - mock_graph[global_idx_2_1][_DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE] + _add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.READ], + mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE], ) - mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.WRITE] + _add_edge( + mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE], + mock_graph[global_idx_2_1][_DAGNodeOperationType.WRITE], ) - mock_graph[global_idx_1_1][_DAGNodeOperationType.READ].add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE] + _add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.READ], + mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE], ) - mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE].add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.WRITE] + _add_edge( + mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE], + mock_graph[global_idx_1_1][_DAGNodeOperationType.WRITE], ) mock_actor_to_candidates = { fake_actor_1: [mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE]], From 6e23a3ab1e5985ae98532d9cabe7d145e5830af3 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 21:41:23 +0000 Subject: [PATCH 094/111] rename idx to dag_idx or local_idx Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 18 +++++---- python/ray/dag/dag_node_operation.py | 37 ++++++++++--------- .../test_execution_schedule_gpu.py | 6 +-- 3 files changed, 32 insertions(+), 29 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 7c3b4f8750230..9b239e288c157 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -113,7 +113,7 @@ def do_exec_tasks( if done: break for operation in schedule: - done = tasks[operation.idx].exec_operation(self, operation.type) + done = tasks[operation.local_idx].exec_operation(self, operation.type) if done: break except Exception: @@ -294,7 +294,9 @@ def __init__( self.input_channels: List[ChannelInterface] = [] self.task_inputs: List[_ExecutableTaskInput] = [] self.resolved_kwargs: Dict[str, Any] = resolved_kwargs - self.idx = task.idx + # A unique index which can be used to index into `idx_to_task` to get + # the corresponding task. + self.dag_idx = task.idx # Reverse map for input_channels: maps an input channel to # its index in input_channels. @@ -1200,26 +1202,26 @@ def _generate_dag_operation_graph_node( for local_idx, exec_task in enumerate(executable_tasks): # Divide a DAG node into three _DAGOperationGraphNodes: READ, COMPUTE, # and WRITE. Each _DAGOperationGraphNode has a _DAGNodeOperation. - idx = exec_task.idx - dag_node = self.idx_to_task[idx].dag_node + dag_idx = exec_task.dag_idx + dag_node = self.idx_to_task[dag_idx].dag_node actor_handle = dag_node._get_actor_handle() requires_nccl = dag_node.type_hint.requires_nccl() read_node = _DAGOperationGraphNode( _DAGNodeOperation(local_idx, _DAGNodeOperationType.READ), - idx, + dag_idx, actor_handle, requires_nccl, ) compute_node = _DAGOperationGraphNode( _DAGNodeOperation(local_idx, _DAGNodeOperationType.COMPUTE), - idx, + dag_idx, actor_handle, requires_nccl, ) write_node = _DAGOperationGraphNode( _DAGNodeOperation(local_idx, _DAGNodeOperationType.WRITE), - idx, + dag_idx, actor_handle, requires_nccl, ) @@ -1299,7 +1301,7 @@ def _build_execution_schedule(self): visited_nodes.add(node) for out_node_idx, out_node_type in node.out_edges: out_node = graph[out_node_idx][out_node_type] - out_node.in_edges.remove((node.idx, node.operation.type)) + out_node.in_edges.remove((node.dag_idx, node.operation.type)) if out_node.in_degree == 0: heapq.heappush( actor_to_candidates[out_node.actor_handle._actor_id], diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 4ed10f4ae6f82..23dc4922ff335 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -21,18 +21,17 @@ class _DAGNodeOperationType(Enum): class _DAGNodeOperation: def __init__( self, - idx: int, + local_idx: int, operation_type: _DAGNodeOperationType, ): """ Args: - idx: The index of the task that this operation belongs to - in the actor's ExecutableTask list. The index is not - the same as bind_index, but there are positive correlations - between the two. + local_idx: The index of the task that this operation belongs to + in the actor's ExecutableTask list. The index is not the same + as bind_index, but there are positive correlations between the two. operation_type: The type of operation to perform. """ - self.idx = idx + self.local_idx = local_idx self.type = operation_type @@ -41,7 +40,7 @@ class _DAGOperationGraphNode: def __init__( self, operation: _DAGNodeOperation, - idx: int, + dag_idx: int, actor_handle: "ray.actor.ActorHandle", requires_nccl: bool, ): @@ -53,11 +52,13 @@ def __init__( Args: operation: The operation that this node performs. The operation can be a READ, COMPUTE, or WRITE operation. - idx: A unique index into the original DAG. + dag_idx: A unique index which can be used to index into + `CompiledDAG.idx_to_task` to get the corresponding task. + dag_node: The DAGNode that this operation belongs to. """ self.operation = operation - self.idx = idx + self.dag_idx = dag_idx self.actor_handle = actor_handle self.requires_nccl = requires_nccl # The in_edges and out_edges are sets of tuples. Each tuple contains @@ -71,16 +72,16 @@ def __init__( def in_degree(self) -> int: return len(self.in_edges) - def __lt__(self, other): + def __lt__(self, other: "_DAGOperationGraphNode"): """ Two _DAGOperationGraphNodes are comparable only when they belong to the same actor. For operations on the same actor, if idx is smaller, the DAGNode to which this operation belongs has a smaller `bind_index`. """ assert self.actor_handle == other.actor_handle - return self.operation.idx < other.operation.idx + return self.operation.local_idx < other.operation.local_idx - def __eq__(self, other): + def __eq__(self, other: "_DAGOperationGraphNode"): """ Two _DAGOperationGraphNodes are comparable only when they belong to the same actor. For operations on the same actor, two operations are equal @@ -88,21 +89,21 @@ def __eq__(self, other): """ assert self.actor_handle == other.actor_handle return ( - self.operation.idx == other.operation.idx + self.operation.local_idx == other.operation.local_idx and self.operation.type == other.operation.type ) def __hash__(self): - return hash((self.operation, self.idx)) + return hash((self.operation, self.dag_idx)) def _add_edge(from_node: _DAGOperationGraphNode, to_node: _DAGOperationGraphNode): """ Add an edge from `from_node` to `to_node`. An edge is a tuple of - the operation's index and type. + the operation's `dag_idx` and type. """ - from_node.out_edges.add((to_node.idx, to_node.operation.type)) - to_node.in_edges.add((from_node.idx, from_node.operation.type)) + from_node.out_edges.add((to_node.dag_idx, to_node.operation.type)) + to_node.in_edges.add((from_node.dag_idx, from_node.operation.type)) def _select_next_nodes( @@ -217,7 +218,7 @@ def _build_dag_node_operation_graph( for _, operation_nodes_list in actor_to_operation_nodes.items(): prev_compute_node = None for operation_nodes in operation_nodes_list: - idx = operation_nodes[0].idx + idx = operation_nodes[0].dag_idx read_node, compute_node, write_node = ( operation_nodes[0], operation_nodes[1], diff --git a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py index e1a123d980d11..2b97832f19885 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py @@ -327,7 +327,7 @@ def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): ): assert len(schedule) == len(expected_schedule) for i, operation in enumerate(schedule): - assert operation.idx == expected_schedule[i][0] + assert operation.local_idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] tensor_cpu = torch.zeros(10, 10) @@ -422,7 +422,7 @@ def test_three_actors_with_nccl_1(ray_start_regular): ): assert len(schedule) == len(expected_schedule) for i, operation in enumerate(schedule): - assert operation.idx == expected_schedule[i][0] + assert operation.local_idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] tensor_cpu = torch.zeros(10, 10) @@ -500,7 +500,7 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): ): assert len(schedule) == len(expected_schedule) for i, operation in enumerate(schedule): - assert operation.idx == expected_schedule[i][0] + assert operation.local_idx == expected_schedule[i][0] assert operation.type == expected_schedule[i][1] tensor_cpu = torch.zeros(10, 10) From 5691e8b5e92d5a55fd9d4dde99d7d30176f51f9b Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 21:59:46 +0000 Subject: [PATCH 095/111] rename global_idx to dag_idx Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 4 +- python/ray/dag/dag_node_operation.py | 31 ++- .../experimental/test_execution_schedule.py | 262 ++++++++---------- 3 files changed, 139 insertions(+), 158 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 9b239e288c157..7661fb6210287 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -1299,8 +1299,8 @@ def _build_execution_schedule(self): continue actor_to_execution_schedule[node.actor_handle].append(node.operation) visited_nodes.add(node) - for out_node_idx, out_node_type in node.out_edges: - out_node = graph[out_node_idx][out_node_type] + for out_node_dag_idx, out_node_type in node.out_edges: + out_node = graph[out_node_dag_idx][out_node_type] out_node.in_edges.remove((node.dag_idx, node.operation.type)) if out_node.in_degree == 0: heapq.heappush( diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 23dc4922ff335..fbedb8118cc07 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -54,8 +54,8 @@ def __init__( can be a READ, COMPUTE, or WRITE operation. dag_idx: A unique index which can be used to index into `CompiledDAG.idx_to_task` to get the corresponding task. - - dag_node: The DAGNode that this operation belongs to. + actor_handle: The actor handle to which this operation belongs. + requires_nccl: Whether this operation requires NCCL. """ self.operation = operation self.dag_idx = dag_idx @@ -75,7 +75,7 @@ def in_degree(self) -> int: def __lt__(self, other: "_DAGOperationGraphNode"): """ Two _DAGOperationGraphNodes are comparable only when they belong to - the same actor. For operations on the same actor, if idx is smaller, + the same actor. For operations on the same actor, if `local_idx` is smaller, the DAGNode to which this operation belongs has a smaller `bind_index`. """ assert self.actor_handle == other.actor_handle @@ -85,7 +85,7 @@ def __eq__(self, other: "_DAGOperationGraphNode"): """ Two _DAGOperationGraphNodes are comparable only when they belong to the same actor. For operations on the same actor, two operations are equal - only when they have the same `idx` and `type`. + only when they have the same `local_idx` and `type`. """ assert self.actor_handle == other.actor_handle return ( @@ -94,6 +94,9 @@ def __eq__(self, other: "_DAGOperationGraphNode"): ) def __hash__(self): + """ + An operation is uniquely identified by its `dag_idx` and type. + """ return hash((self.operation, self.dag_idx)) @@ -169,8 +172,8 @@ def _select_next_nodes( # An NCCL write node is picked. NCCL is a blocking operation, so we need to pick all # the corresponding NCCL read nodes to avoid a deadlock. for downstream_node_metadata in first_nccl_node.out_edges: - global_idx, op_type = downstream_node_metadata[0], downstream_node_metadata[1] - downstream_node = graph[global_idx][op_type] + dag_idx, op_type = downstream_node_metadata[0], downstream_node_metadata[1] + downstream_node = graph[dag_idx][op_type] assert downstream_node.operation.type == _DAGNodeOperationType.READ next_nodes.append(downstream_node) assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) @@ -218,7 +221,7 @@ def _build_dag_node_operation_graph( for _, operation_nodes_list in actor_to_operation_nodes.items(): prev_compute_node = None for operation_nodes in operation_nodes_list: - idx = operation_nodes[0].dag_idx + dag_idx = operation_nodes[0].dag_idx read_node, compute_node, write_node = ( operation_nodes[0], operation_nodes[1], @@ -233,8 +236,8 @@ def _build_dag_node_operation_graph( if prev_compute_node is not None: _add_edge(prev_compute_node, compute_node) prev_compute_node = compute_node - assert idx not in graph - graph[idx] = { + assert dag_idx not in graph + graph[dag_idx] = { _DAGNodeOperationType.READ: read_node, _DAGNodeOperationType.COMPUTE: compute_node, _DAGNodeOperationType.WRITE: write_node, @@ -244,18 +247,18 @@ def _build_dag_node_operation_graph( from ray.dag import ClassMethodNode, MultiOutputNode # Add an edge from WRITE of the writer task to READ of the reader task. - for idx, task in idx_to_task.items(): + for dag_idx, task in idx_to_task.items(): if not isinstance(task.dag_node, ClassMethodNode): # The graph is used to generate an execution schedule for each actor. # The edge from the InputNode has no impact on the final execution # schedule. continue - for downstream_idx in task.downstream_node_idxs: - downstream_dag_node = idx_to_task[downstream_idx].dag_node + for downstream_dag_idx in task.downstream_node_idxs: + downstream_dag_node = idx_to_task[downstream_dag_idx].dag_node if isinstance(downstream_dag_node, MultiOutputNode): continue _add_edge( - graph[idx][_DAGNodeOperationType.WRITE], - graph[downstream_idx][_DAGNodeOperationType.READ], + graph[dag_idx][_DAGNodeOperationType.WRITE], + graph[downstream_dag_idx][_DAGNodeOperationType.READ], ) return graph diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index feae04a2f4539..c2c06ed6cfd09 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -30,12 +30,12 @@ def mock_init(self): pass -def generate_dag_graph_nodes(local_idx, global_idx, actor_handle, requires_nccl): +def generate_dag_graph_nodes(local_idx, dag_idx, actor_handle, requires_nccl): graph_nodes = {} for op_type in _DAGNodeOperationType: graph_nodes[op_type] = _DAGOperationGraphNode( _DAGNodeOperation(local_idx, op_type), - global_idx, + dag_idx, actor_handle, requires_nccl, ) @@ -47,7 +47,7 @@ class TestSelectNextNodes: Test whether `_select_next_nodes` function selects the next nodes for topological sort to generate execution schedule correctly. - global_idx: Each DAG node has a unique global index. + dag_idx: Each DAG node has a unique global index. local_idx: The DAG node's index in the actor's `executable_tasks` list. """ @@ -67,19 +67,19 @@ def test_two_candidates_on_same_actor(self): fake_actor = "fake_actor" # The DAG node has a global index of 1, and its index in the # actor's `executable_tasks` list is 0. - global_idx_1 = 1 + dag_idx_1 = 1 dag_node_1 = _DAGOperationGraphNode( _DAGNodeOperation(0, _DAGNodeOperationType.READ), - global_idx_1, + dag_idx_1, fake_actor, False, ) # The DAG node has a global index of 2, and its index in the # actor's `executable_tasks` list is 1. - global_idx_2 = 2 + dag_idx_2 = 2 dag_node_2 = _DAGOperationGraphNode( _DAGNodeOperation(1, _DAGNodeOperationType.READ), - global_idx_2, + dag_idx_2, fake_actor, False, ) @@ -107,38 +107,38 @@ def test_only_one_nccl_write(self, monkeypatch): execution schedule. """ monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) - fake_actor_1, global_idx_1, local_idx_1 = ActorHandle("fake_actor_1"), 1, 0 - fake_actor_2, global_idx_2, local_idx_2 = ActorHandle("fake_actor_2"), 2, 0 + fake_actor_1, dag_idx_1, local_idx_1 = ActorHandle("fake_actor_1"), 1, 0 + fake_actor_2, dag_idx_2, local_idx_2 = ActorHandle("fake_actor_2"), 2, 0 mock_graph = { - global_idx_1: generate_dag_graph_nodes( - local_idx_1, global_idx_1, fake_actor_1, True + dag_idx_1: generate_dag_graph_nodes( + local_idx_1, dag_idx_1, fake_actor_1, True ), - global_idx_2: generate_dag_graph_nodes( - local_idx_2, global_idx_2, fake_actor_2, False + dag_idx_2: generate_dag_graph_nodes( + local_idx_2, dag_idx_2, fake_actor_2, False ), } - del mock_graph[global_idx_1][_DAGNodeOperationType.READ] - del mock_graph[global_idx_1][_DAGNodeOperationType.COMPUTE] + del mock_graph[dag_idx_1][_DAGNodeOperationType.READ] + del mock_graph[dag_idx_1][_DAGNodeOperationType.COMPUTE] _add_edge( - mock_graph[global_idx_1][_DAGNodeOperationType.WRITE], - mock_graph[global_idx_2][_DAGNodeOperationType.READ], + mock_graph[dag_idx_1][_DAGNodeOperationType.WRITE], + mock_graph[dag_idx_2][_DAGNodeOperationType.READ], ) _add_edge( - mock_graph[global_idx_2][_DAGNodeOperationType.READ], - mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE], + mock_graph[dag_idx_2][_DAGNodeOperationType.READ], + mock_graph[dag_idx_2][_DAGNodeOperationType.COMPUTE], ) _add_edge( - mock_graph[global_idx_2][_DAGNodeOperationType.COMPUTE], - mock_graph[global_idx_2][_DAGNodeOperationType.WRITE], + mock_graph[dag_idx_2][_DAGNodeOperationType.COMPUTE], + mock_graph[dag_idx_2][_DAGNodeOperationType.WRITE], ) mock_actor_to_candidates = { - fake_actor_1: [mock_graph[global_idx_1][_DAGNodeOperationType.WRITE]], + fake_actor_1: [mock_graph[dag_idx_1][_DAGNodeOperationType.WRITE]], } next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) assert len(next_nodes) == 2 - assert next_nodes[0] == mock_graph[global_idx_1][_DAGNodeOperationType.WRITE] - assert next_nodes[1] == mock_graph[global_idx_2][_DAGNodeOperationType.READ] + assert next_nodes[0] == mock_graph[dag_idx_1][_DAGNodeOperationType.WRITE] + assert next_nodes[1] == mock_graph[dag_idx_2][_DAGNodeOperationType.READ] def test_two_nccl_writes(self, monkeypatch): """ @@ -158,71 +158,67 @@ def test_two_nccl_writes(self, monkeypatch): monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) fake_actor_1 = ActorHandle("fake_actor_1") - global_idx_1_0, local_idx_1_0 = 1, 0 - global_idx_1_1, local_idx_1_1 = 3, 1 + dag_idx_1_0, local_idx_1_0 = 1, 0 + dag_idx_1_1, local_idx_1_1 = 3, 1 fake_actor_2 = ActorHandle("fake_actor_2") - global_idx_2_0, local_idx_2_0 = 2, 0 - global_idx_2_1, local_idx_2_1 = 4, 1 + dag_idx_2_0, local_idx_2_0 = 2, 0 + dag_idx_2_1, local_idx_2_1 = 4, 1 # Run the test 10 times to ensure that the result of `_select_next_nodes` # is deterministic. for _ in range(20): mock_graph = { - global_idx_1_0: generate_dag_graph_nodes( - local_idx_1_0, global_idx_1_0, fake_actor_1, True + dag_idx_1_0: generate_dag_graph_nodes( + local_idx_1_0, dag_idx_1_0, fake_actor_1, True ), - global_idx_1_1: generate_dag_graph_nodes( - local_idx_1_1, global_idx_1_1, fake_actor_1, False + dag_idx_1_1: generate_dag_graph_nodes( + local_idx_1_1, dag_idx_1_1, fake_actor_1, False ), - global_idx_2_0: generate_dag_graph_nodes( - local_idx_2_0, global_idx_2_0, fake_actor_2, True + dag_idx_2_0: generate_dag_graph_nodes( + local_idx_2_0, dag_idx_2_0, fake_actor_2, True ), - global_idx_2_1: generate_dag_graph_nodes( - local_idx_2_1, global_idx_2_1, fake_actor_2, False + dag_idx_2_1: generate_dag_graph_nodes( + local_idx_2_1, dag_idx_2_1, fake_actor_2, False ), } - del mock_graph[global_idx_1_0][_DAGNodeOperationType.READ] - del mock_graph[global_idx_1_0][_DAGNodeOperationType.COMPUTE] - del mock_graph[global_idx_2_0][_DAGNodeOperationType.READ] - del mock_graph[global_idx_2_0][_DAGNodeOperationType.COMPUTE] + del mock_graph[dag_idx_1_0][_DAGNodeOperationType.READ] + del mock_graph[dag_idx_1_0][_DAGNodeOperationType.COMPUTE] + del mock_graph[dag_idx_2_0][_DAGNodeOperationType.READ] + del mock_graph[dag_idx_2_0][_DAGNodeOperationType.COMPUTE] _add_edge( - mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE], - mock_graph[global_idx_2_1][_DAGNodeOperationType.READ], + mock_graph[dag_idx_1_0][_DAGNodeOperationType.WRITE], + mock_graph[dag_idx_2_1][_DAGNodeOperationType.READ], ) _add_edge( - mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE], - mock_graph[global_idx_1_1][_DAGNodeOperationType.READ], + mock_graph[dag_idx_2_0][_DAGNodeOperationType.WRITE], + mock_graph[dag_idx_1_1][_DAGNodeOperationType.READ], ) _add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.READ], - mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE], + mock_graph[dag_idx_2_1][_DAGNodeOperationType.READ], + mock_graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE], ) _add_edge( - mock_graph[global_idx_2_1][_DAGNodeOperationType.COMPUTE], - mock_graph[global_idx_2_1][_DAGNodeOperationType.WRITE], + mock_graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE], + mock_graph[dag_idx_2_1][_DAGNodeOperationType.WRITE], ) _add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.READ], - mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE], + mock_graph[dag_idx_1_1][_DAGNodeOperationType.READ], + mock_graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE], ) _add_edge( - mock_graph[global_idx_1_1][_DAGNodeOperationType.COMPUTE], - mock_graph[global_idx_1_1][_DAGNodeOperationType.WRITE], + mock_graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE], + mock_graph[dag_idx_1_1][_DAGNodeOperationType.WRITE], ) mock_actor_to_candidates = { - fake_actor_1: [mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE]], - fake_actor_2: [mock_graph[global_idx_2_0][_DAGNodeOperationType.WRITE]], + fake_actor_1: [mock_graph[dag_idx_1_0][_DAGNodeOperationType.WRITE]], + fake_actor_2: [mock_graph[dag_idx_2_0][_DAGNodeOperationType.WRITE]], } next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) assert len(next_nodes) == 2 - assert ( - next_nodes[0] == mock_graph[global_idx_1_0][_DAGNodeOperationType.WRITE] - ) - assert ( - next_nodes[1] == mock_graph[global_idx_2_1][_DAGNodeOperationType.READ] - ) + assert next_nodes[0] == mock_graph[dag_idx_1_0][_DAGNodeOperationType.WRITE] + assert next_nodes[1] == mock_graph[dag_idx_2_1][_DAGNodeOperationType.READ] class TestBuildDAGNodeOperationGraph: @@ -235,7 +231,7 @@ class TestBuildDAGNodeOperationGraph: def check_edges_between_read_compute_write( self, graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], - global_idx: int, + dag_idx: int, expected_num_edges: List[Tuple[int, int]], ): """ @@ -244,54 +240,54 @@ def check_edges_between_read_compute_write( Args: graph: The operation graph generated by `_build_dag_node_operation_graph`. - global_idx: The global index of the task used to access the task in + dag_idx: The global index of the task used to access the task in `idx_to_task`. expected_num_edges: A list of tuples where each tuple contains the expected number of in-edges and out-edges for READ, COMPUTE, and WRITE operations. """ assert len(expected_num_edges) == 3 - assert len(graph[global_idx]) == 3 - read_node = graph[global_idx][_DAGNodeOperationType.READ] - compute_node = graph[global_idx][_DAGNodeOperationType.COMPUTE] - write_node = graph[global_idx][_DAGNodeOperationType.WRITE] + assert len(graph[dag_idx]) == 3 + read_node = graph[dag_idx][_DAGNodeOperationType.READ] + compute_node = graph[dag_idx][_DAGNodeOperationType.COMPUTE] + write_node = graph[dag_idx][_DAGNodeOperationType.WRITE] for idx, node in enumerate([read_node, compute_node, write_node]): assert node.in_degree == expected_num_edges[idx][0] assert len(node.out_edges) == expected_num_edges[idx][1] - assert (global_idx, _DAGNodeOperationType.COMPUTE) in read_node.out_edges - assert (global_idx, _DAGNodeOperationType.READ) in compute_node.in_edges - assert (global_idx, _DAGNodeOperationType.WRITE) in compute_node.out_edges - assert (global_idx, _DAGNodeOperationType.COMPUTE) in write_node.in_edges + assert (dag_idx, _DAGNodeOperationType.COMPUTE) in read_node.out_edges + assert (dag_idx, _DAGNodeOperationType.READ) in compute_node.in_edges + assert (dag_idx, _DAGNodeOperationType.WRITE) in compute_node.out_edges + assert (dag_idx, _DAGNodeOperationType.COMPUTE) in write_node.in_edges def check_edge_between_writer_and_reader( self, graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], - writer_global_idx: int, - reader_global_idx: int, + writer_dag_idx: int, + reader_dag_idx: int, ): """ Check whether the edge from writer's WRITE to reader's READ operation is added. Args: graph: The operation graph generated by `_build_dag_node_operation_graph`. - writer_global_idx: The global index of the task used to access the task + writer_dag_idx: The index of the task used to access the task that the writer belongs to in `idx_to_task`. - reader_global_idx: The global index of the task used to access the task + reader_dag_idx: The index of the task used to access the task that the reader belongs to in `idx_to_task`. """ - write_node = graph[writer_global_idx][_DAGNodeOperationType.WRITE] - read_node = graph[reader_global_idx][_DAGNodeOperationType.READ] + write_node = graph[writer_dag_idx][_DAGNodeOperationType.WRITE] + read_node = graph[reader_dag_idx][_DAGNodeOperationType.READ] - assert (reader_global_idx, _DAGNodeOperationType.READ) in write_node.out_edges - assert (writer_global_idx, _DAGNodeOperationType.WRITE) in read_node.in_edges + assert (reader_dag_idx, _DAGNodeOperationType.READ) in write_node.out_edges + assert (writer_dag_idx, _DAGNodeOperationType.WRITE) in read_node.in_edges def check_edge_between_compute_nodes( self, graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], - global_idx_1: int, - global_idx_2: int, + dag_idx_1: int, + dag_idx_2: int, ): """ Check whether the edge from COMPUTE with `bind_index` i to COMPUTE with @@ -299,18 +295,18 @@ def check_edge_between_compute_nodes( Args: graph: The operation graph generated by `_build_dag_node_operation_graph`. - global_idx_1: The global index of the task used to access the task in + dag_idx_1: The index of the task used to access the task in `idx_to_task`. - global_idx_2: The global index of the task used to access the task in + dag_idx_2: The index of the task used to access the task in `idx_to_task`. Note that both tasks belong to the same actor, and the `bind_index` of the second task is equal to the `bind_index` of the first task plus one. """ - compute_node_1 = graph[global_idx_1][_DAGNodeOperationType.COMPUTE] - compute_node_2 = graph[global_idx_2][_DAGNodeOperationType.COMPUTE] + compute_node_1 = graph[dag_idx_1][_DAGNodeOperationType.COMPUTE] + compute_node_2 = graph[dag_idx_2][_DAGNodeOperationType.COMPUTE] - assert (global_idx_2, _DAGNodeOperationType.COMPUTE) in compute_node_1.out_edges - assert (global_idx_1, _DAGNodeOperationType.COMPUTE) in compute_node_2.in_edges + assert (dag_idx_2, _DAGNodeOperationType.COMPUTE) in compute_node_1.out_edges + assert (dag_idx_1, _DAGNodeOperationType.COMPUTE) in compute_node_2.in_edges def test_edges_between_read_compute_write(self, monkeypatch): """ @@ -329,19 +325,17 @@ def test_edges_between_read_compute_write(self, monkeypatch): } fake_actor = "fake_actor" - global_idx = 1 + dag_idx = 1 actor_to_operation_nodes = { fake_actor: [ - list( - generate_dag_graph_nodes(0, global_idx, fake_actor, False).values() - ) + list(generate_dag_graph_nodes(0, dag_idx, fake_actor, False).values()) ] } graph = _build_dag_node_operation_graph(idx_to_task, actor_to_operation_nodes) assert len(graph) == 1 self.check_edges_between_read_compute_write( - graph, global_idx, [(0, 1), (1, 1), (1, 0)] + graph, dag_idx, [(0, 1), (1, 1), (1, 0)] ) def test_edge_between_writer_and_reader(self, monkeypatch): @@ -354,8 +348,8 @@ def test_edge_between_writer_and_reader(self, monkeypatch): monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) - fake_actor_1, global_idx_1 = "fake_actor_1", 1 - fake_actor_2, global_idx_2 = "fake_actor_2", 2 + fake_actor_1, dag_idx_1 = "fake_actor_1", 1 + fake_actor_2, dag_idx_2 = "fake_actor_2", 2 idx_to_task = { 0: CompiledTask(0, InputNode()), 1: CompiledTask(1, ClassMethodNode()), @@ -367,16 +361,12 @@ def test_edge_between_writer_and_reader(self, monkeypatch): actor_to_operation_nodes = { fake_actor_1: [ list( - generate_dag_graph_nodes( - 0, global_idx_1, fake_actor_1, False - ).values() + generate_dag_graph_nodes(0, dag_idx_1, fake_actor_1, False).values() ) ], fake_actor_2: [ list( - generate_dag_graph_nodes( - 0, global_idx_2, fake_actor_2, False - ).values() + generate_dag_graph_nodes(0, dag_idx_2, fake_actor_2, False).values() ) ], } @@ -384,12 +374,12 @@ def test_edge_between_writer_and_reader(self, monkeypatch): assert len(graph) == 2 self.check_edges_between_read_compute_write( - graph, global_idx_1, [(0, 1), (1, 1), (1, 1)] + graph, dag_idx_1, [(0, 1), (1, 1), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, global_idx_2, [(1, 1), (1, 1), (1, 0)] + graph, dag_idx_2, [(1, 1), (1, 1), (1, 0)] ) - self.check_edge_between_writer_and_reader(graph, global_idx_1, global_idx_2) + self.check_edge_between_writer_and_reader(graph, dag_idx_1, dag_idx_2) def test_edge_between_compute_nodes(self, monkeypatch): """ @@ -403,26 +393,22 @@ def test_edge_between_compute_nodes(self, monkeypatch): monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) fake_actor = "fake_actor" - global_idx_1, global_idx_2 = 1, 2 + dag_idx_1, dag_idx_2 = 1, 2 idx_to_task = { 0: CompiledTask(0, InputNode()), - global_idx_1: CompiledTask(global_idx_1, ClassMethodNode()), - global_idx_2: CompiledTask(global_idx_2, ClassMethodNode()), + dag_idx_1: CompiledTask(dag_idx_1, ClassMethodNode()), + dag_idx_2: CompiledTask(dag_idx_2, ClassMethodNode()), 3: CompiledTask(3, MultiOutputNode()), } - idx_to_task[global_idx_1].downstream_node_idxs = {global_idx_2: fake_actor} + idx_to_task[dag_idx_1].downstream_node_idxs = {dag_idx_2: fake_actor} actor_to_operation_nodes = { fake_actor: [ list( - generate_dag_graph_nodes( - 0, global_idx_1, fake_actor, False - ).values() + generate_dag_graph_nodes(0, dag_idx_1, fake_actor, False).values() ), list( - generate_dag_graph_nodes( - 1, global_idx_2, fake_actor, False - ).values() + generate_dag_graph_nodes(1, dag_idx_2, fake_actor, False).values() ), ], } @@ -430,13 +416,13 @@ def test_edge_between_compute_nodes(self, monkeypatch): assert len(graph) == 2 self.check_edges_between_read_compute_write( - graph, global_idx_1, [(0, 1), (1, 2), (1, 1)] + graph, dag_idx_1, [(0, 1), (1, 2), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, global_idx_2, [(1, 1), (2, 1), (1, 0)] + graph, dag_idx_2, [(1, 1), (2, 1), (1, 0)] ) - self.check_edge_between_writer_and_reader(graph, global_idx_1, global_idx_2) - self.check_edge_between_compute_nodes(graph, global_idx_1, global_idx_2) + self.check_edge_between_writer_and_reader(graph, dag_idx_1, dag_idx_2) + self.check_edge_between_compute_nodes(graph, dag_idx_1, dag_idx_2) def test_two_actors(self, monkeypatch): """ @@ -451,43 +437,35 @@ def test_two_actors(self, monkeypatch): monkeypatch.setattr(ClassMethodNode, "__init__", mock_init) monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) - fake_actor_1, global_idx_1, global_idx_3 = "fake_actor_1", 1, 3 - fake_actor_2, global_idx_2, global_idx_4 = "fake_actor_2", 2, 4 + fake_actor_1, dag_idx_1, dag_idx_3 = "fake_actor_1", 1, 3 + fake_actor_2, dag_idx_2, dag_idx_4 = "fake_actor_2", 2, 4 idx_to_task = { 0: CompiledTask(0, InputNode()), - global_idx_1: CompiledTask(global_idx_1, ClassMethodNode()), - global_idx_2: CompiledTask(global_idx_2, ClassMethodNode()), - global_idx_3: CompiledTask(global_idx_3, ClassMethodNode()), - global_idx_4: CompiledTask(global_idx_4, ClassMethodNode()), + dag_idx_1: CompiledTask(dag_idx_1, ClassMethodNode()), + dag_idx_2: CompiledTask(dag_idx_2, ClassMethodNode()), + dag_idx_3: CompiledTask(dag_idx_3, ClassMethodNode()), + dag_idx_4: CompiledTask(dag_idx_4, ClassMethodNode()), 5: CompiledTask(5, MultiOutputNode()), } - idx_to_task[global_idx_1].downstream_node_idxs = {global_idx_4: fake_actor_2} - idx_to_task[global_idx_2].downstream_node_idxs = {global_idx_3: fake_actor_1} + idx_to_task[dag_idx_1].downstream_node_idxs = {dag_idx_4: fake_actor_2} + idx_to_task[dag_idx_2].downstream_node_idxs = {dag_idx_3: fake_actor_1} actor_to_operation_nodes = { fake_actor_1: [ list( - generate_dag_graph_nodes( - 0, global_idx_1, fake_actor_1, False - ).values() + generate_dag_graph_nodes(0, dag_idx_1, fake_actor_1, False).values() ), list( - generate_dag_graph_nodes( - 1, global_idx_3, fake_actor_1, False - ).values() + generate_dag_graph_nodes(1, dag_idx_3, fake_actor_1, False).values() ), ], fake_actor_2: [ list( - generate_dag_graph_nodes( - 0, global_idx_2, fake_actor_2, False - ).values() + generate_dag_graph_nodes(0, dag_idx_2, fake_actor_2, False).values() ), list( - generate_dag_graph_nodes( - 1, global_idx_4, fake_actor_2, False - ).values() + generate_dag_graph_nodes(1, dag_idx_4, fake_actor_2, False).values() ), ], } @@ -495,19 +473,19 @@ def test_two_actors(self, monkeypatch): assert len(graph) == 4 self.check_edges_between_read_compute_write( - graph, global_idx_1, [(0, 1), (1, 2), (1, 1)] + graph, dag_idx_1, [(0, 1), (1, 2), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, global_idx_2, [(0, 1), (1, 2), (1, 1)] + graph, dag_idx_2, [(0, 1), (1, 2), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, global_idx_3, [(1, 1), (2, 1), (1, 0)] + graph, dag_idx_3, [(1, 1), (2, 1), (1, 0)] ) self.check_edges_between_read_compute_write( - graph, global_idx_4, [(1, 1), (2, 1), (1, 0)] + graph, dag_idx_4, [(1, 1), (2, 1), (1, 0)] ) - self.check_edge_between_writer_and_reader(graph, global_idx_1, global_idx_4) - self.check_edge_between_writer_and_reader(graph, global_idx_2, global_idx_3) + self.check_edge_between_writer_and_reader(graph, dag_idx_1, dag_idx_4) + self.check_edge_between_writer_and_reader(graph, dag_idx_2, dag_idx_3) if __name__ == "__main__": From a30f493d4aee57916444d696cc942ac40b680070 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 22:10:32 +0000 Subject: [PATCH 096/111] update comments for return in _build_dag_node_operation_graph Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index fbedb8118cc07..9fca7c31eee10 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -210,10 +210,10 @@ def _build_dag_node_operation_graph( the inner list, the order of operations is READ, COMPUTE, and WRITE. Returns: - A graph where each node is a _DAGOperationGraphNode. The key is the index - of the task in idx_to_task, and the value is a dictionary that maps the - _DAGNodeOperationType (READ, COMPUTE, or WRITE) to the corresponding - _DAGOperationGraphNode. + A graph where each node is a _DAGOperationGraphNode. The key is `dag_idx`, + the index to retrieve its task from `idx_to_task`, and the value is a + dictionary that maps the _DAGNodeOperationType (READ, COMPUTE, or WRITE) + to the corresponding _DAGOperationGraphNode """ assert idx_to_task graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]] = {} From 44abfbbe7d1b82710fadedf04df97d0e82954688 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 22:38:54 +0000 Subject: [PATCH 097/111] move topological sort to dag_node_operation.py Signed-off-by: Kai-Hsun Chen --- python/ray/dag/compiled_dag_node.py | 57 ++-------------------- python/ray/dag/dag_node_operation.py | 73 ++++++++++++++++++++++++++++ 2 files changed, 76 insertions(+), 54 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index 7661fb6210287..8bc052a7e35b3 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -41,12 +41,11 @@ _DAGNodeOperation, _DAGNodeOperationType, _DAGOperationGraphNode, - _select_next_nodes, _build_dag_node_operation_graph, + _generate_actor_to_execution_schedule, ) from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy -import heapq # Holds the input arguments for an accelerated DAG node. @@ -1257,63 +1256,13 @@ def _build_execution_schedule(self): actor_to_execution_schedule: A dictionary that maps an actor handle to the execution schedule which is a list of operations to be executed. """ - # Mapping from the actor handle to the execution schedule which is a list - # of operations to be executed. - actor_to_execution_schedule: Dict[ - "ray.actor.ActorHandle", List[_DAGNodeOperation] - ] = defaultdict(list) - # Step 1: Build a graph of _DAGOperationGraphNode actor_to_operation_nodes = self._generate_dag_operation_graph_node() graph = _build_dag_node_operation_graph( self.idx_to_task, actor_to_operation_nodes ) - - # A dictionary mapping an actor id to a list of candidate nodes. The list - # is maintained as a priority queue, so the head of the queue, i.e., - # `candidates[0]`, is the node with the smallest `bind_index`. - actor_to_candidates: Dict[ - "ray._raylet.ActorID", List[_DAGOperationGraphNode] - ] = defaultdict(list) - for _, node_dict in graph.items(): - for _, node in node_dict.items(): - # A node with a zero in-degree edge means all of its dependencies - # have been satisfied, including both data and control dependencies. - # Therefore, it is a candidate for execution. - if node.in_degree == 0: - heapq.heappush( - actor_to_candidates[node.actor_handle._actor_id], node - ) - - visited_nodes = set() - - # Step 2: Topological sort - while actor_to_candidates: - # The function `_select_next_nodes` will pop a candidate node from - # `actor_to_candidates` and return a list of nodes that can be executed - # in the next step. If multiple nodes are returned, only the NCCL write - # node is popped in this iteration. - nodes = _select_next_nodes(actor_to_candidates, graph) - for node in nodes: - if node in visited_nodes: - continue - actor_to_execution_schedule[node.actor_handle].append(node.operation) - visited_nodes.add(node) - for out_node_dag_idx, out_node_type in node.out_edges: - out_node = graph[out_node_dag_idx][out_node_type] - out_node.in_edges.remove((node.dag_idx, node.operation.type)) - if out_node.in_degree == 0: - heapq.heappush( - actor_to_candidates[out_node.actor_handle._actor_id], - out_node, - ) - - delete_keys = [] - for actor_id, candidates in actor_to_candidates.items(): - if len(candidates) == 0: - delete_keys.append(actor_id) - for key in delete_keys: - del actor_to_candidates[key] + # Step 2: Generate an execution schedule for each actor using topological sort + actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) return actor_to_execution_schedule def _detect_deadlock(self) -> bool: diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 9fca7c31eee10..1baa2e883bcfd 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -3,6 +3,7 @@ from typing import Set, Tuple, List, Dict, Optional import ray import heapq +from collections import defaultdict class _DAGNodeOperationType(Enum): @@ -262,3 +263,75 @@ def _build_dag_node_operation_graph( graph[downstream_dag_idx][_DAGNodeOperationType.READ], ) return graph + + +def _generate_actor_to_execution_schedule( + graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]] +): + """ + Generate an execution schedule for each actor. The schedule is a list of + operations to be executed. The function uses a topological sort algorithm + to generate the schedule. + + Args: + graph: A graph where each node is a _DAGOperationGraphNode. The key is + `dag_idx`, the index to retrieve its task from `idx_to_task`, and + the value is a dictionary that maps the _DAGNodeOperationType (READ, + COMPUTE, or WRITE) to the corresponding _DAGOperationGraphNode. It is + generated by `_build_dag_node_operation_graph`. + + Returns: + actor_to_execution_schedule: A dictionary that maps an actor handle to + the execution schedule which is a list of operations to be executed. + """ + + # Mapping from the actor handle to the execution schedule which is a list + # of operations to be executed. + actor_to_execution_schedule: Dict[ + "ray.actor.ActorHandle", List[_DAGNodeOperation] + ] = defaultdict(list) + + # A dictionary mapping an actor id to a list of candidate nodes. The list + # is maintained as a priority queue, so the head of the queue, i.e., + # `candidates[0]`, is the node with the smallest `bind_index`. + actor_to_candidates: Dict[ + "ray._raylet.ActorID", List[_DAGOperationGraphNode] + ] = defaultdict(list) + for _, node_dict in graph.items(): + for _, node in node_dict.items(): + # A node with a zero in-degree edge means all of its dependencies + # have been satisfied, including both data and control dependencies. + # Therefore, it is a candidate for execution. + if node.in_degree == 0: + heapq.heappush(actor_to_candidates[node.actor_handle._actor_id], node) + + visited_nodes = set() + + # Topological sort + while actor_to_candidates: + # The function `_select_next_nodes` will pop a candidate node from + # `actor_to_candidates` and return a list of nodes that can be executed + # in the next step. If multiple nodes are returned, only the NCCL write + # node is popped in this iteration. + nodes = _select_next_nodes(actor_to_candidates, graph) + for node in nodes: + if node in visited_nodes: + continue + actor_to_execution_schedule[node.actor_handle].append(node.operation) + visited_nodes.add(node) + for out_node_dag_idx, out_node_type in node.out_edges: + out_node = graph[out_node_dag_idx][out_node_type] + out_node.in_edges.remove((node.dag_idx, node.operation.type)) + if out_node.in_degree == 0: + heapq.heappush( + actor_to_candidates[out_node.actor_handle._actor_id], + out_node, + ) + + delete_keys = [] + for actor_id, candidates in actor_to_candidates.items(): + if len(candidates) == 0: + delete_keys.append(actor_id) + for key in delete_keys: + del actor_to_candidates[key] + return actor_to_execution_schedule From bca959b33fe0109f6c7578073276fb1f08a6e1a3 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Fri, 9 Aug 2024 23:59:43 +0000 Subject: [PATCH 098/111] add tests for _generate_actor_to_execution_schedule Signed-off-by: Kai-Hsun Chen --- .../experimental/test_execution_schedule.py | 293 ++++++++++++++++++ 1 file changed, 293 insertions(+) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index c2c06ed6cfd09..0c2b0f6789e7c 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -13,6 +13,7 @@ _select_next_nodes, _build_dag_node_operation_graph, _add_edge, + _generate_actor_to_execution_schedule, ) from ray.dag.compiled_dag_node import CompiledTask from typing import List, Dict, Tuple @@ -488,6 +489,298 @@ def test_two_actors(self, monkeypatch): self.check_edge_between_writer_and_reader(graph, dag_idx_2, dag_idx_3) +class TestGenerateActorToExecutionSchedule: + """ + Test whether `_generate_actor_to_execution_schedule` function generates the + correct execution schedule for each actor. + """ + + def add_edge_between_read_compute_write( + self, operations: Dict[_DAGNodeOperationType, _DAGOperationGraphNode] + ): + """ + Add edges between READ and COMPUTE, and between COMPUTE and WRITE operations + on the same actor. + + Args: + operations: A dictionary where the key is the operation type and the value + is the operation node. + """ + assert len(operations) == 3 + _add_edge( + operations[_DAGNodeOperationType.READ], + operations[_DAGNodeOperationType.COMPUTE], + ) + _add_edge( + operations[_DAGNodeOperationType.COMPUTE], + operations[_DAGNodeOperationType.WRITE], + ) + + def add_data_dependeny( + self, + writer_operations: Dict[_DAGNodeOperationType, _DAGOperationGraphNode], + reader_operations: Dict[_DAGNodeOperationType, _DAGOperationGraphNode], + ): + """ + Add a data dependency between the WRITE operation of the writer and the READ + operation of the reader. + + Args: + writer_operations: A dictionary where the key is the operation type and the + value is the operation node of the writer. + reader_operations: A dictionary where the key is the operation type and the + value is the operation node of the reader. + """ + _add_edge( + writer_operations[_DAGNodeOperationType.WRITE], + reader_operations[_DAGNodeOperationType.READ], + ) + + def add_control_dependency( + self, + operations_1: Dict[_DAGNodeOperationType, _DAGOperationGraphNode], + operations_2: Dict[_DAGNodeOperationType, _DAGOperationGraphNode], + ): + """ + Add a control dependency between the COMPUTE operation of the task with + bind_index i and the COMPUTE operation of the task with bind_index i+1 + on the same actor. + + Args: + operations_1: A dictionary where the key is the operation type and the value + is the operation node of the task with bind_index i. + operations_2: A dictionary where the key is the operation type and the value + is the operation node of the task with bind_index i+1. + """ + _add_edge( + operations_1[_DAGNodeOperationType.COMPUTE], + operations_2[_DAGNodeOperationType.COMPUTE], + ) + + def test_single_actor_1(self, monkeypatch): + """ + driver -> fake_actor.op (dag_idx_1) -> fake_actor.op (dag_idx_2) -> driver + + Test the case where there is only one actor and no NCCL operations. + Because there is no NCCL operation, all operations with smaller + `bind_index` should be executed before the operations with larger + `bind_index` on the same actor. + """ + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + + fake_actor = ActorHandle("fake_actor") + dag_idx_1, local_idx_1 = 1, 0 + dag_idx_2, local_idx_2 = 2, 1 + graph = { + dag_idx_1: generate_dag_graph_nodes( + local_idx_1, dag_idx_1, fake_actor, False + ), + dag_idx_2: generate_dag_graph_nodes( + local_idx_2, dag_idx_2, fake_actor, False + ), + } + self.add_edge_between_read_compute_write(graph[dag_idx_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_2]) + self.add_data_dependeny(graph[dag_idx_1], graph[dag_idx_2]) + self.add_control_dependency(graph[dag_idx_1], graph[dag_idx_2]) + + actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) + assert len(actor_to_execution_schedule) == 1 + assert len(actor_to_execution_schedule[fake_actor]) == 6 + assert actor_to_execution_schedule[fake_actor] == [ + graph[dag_idx_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2][_DAGNodeOperationType.WRITE].operation, + ] + + def test_single_actor_2(self, monkeypatch): + """ + driver -> fake_actor.op (dag_idx_1) -> fake_actor.op (dag_idx_2) -> driver + | | + -> fake_actor.op (dag_idx_3) - + + When the `dad_idx_1.WRITE` operation is picked, both `dag_idx_2.READ` and + `dag_idx_3.READ` operations should be zero in-degree. In this case, the one + with the smaller `bind_index` should be selected first. That is, + `dag_idx_2.READ` should be selected first. + """ + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + + fake_actor = ActorHandle("fake_actor") + dag_idx_1, local_idx_1 = 1, 0 + dag_idx_2, local_idx_2 = 2, 1 + dag_idx_3, local_idx_3 = 3, 2 + + graph = { + dag_idx_1: generate_dag_graph_nodes( + local_idx_1, dag_idx_1, fake_actor, False + ), + dag_idx_2: generate_dag_graph_nodes( + local_idx_2, dag_idx_2, fake_actor, False + ), + dag_idx_3: generate_dag_graph_nodes( + local_idx_3, dag_idx_3, fake_actor, False + ), + } + self.add_edge_between_read_compute_write(graph[dag_idx_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_2]) + self.add_edge_between_read_compute_write(graph[dag_idx_3]) + self.add_data_dependeny(graph[dag_idx_1], graph[dag_idx_2]) + self.add_data_dependeny(graph[dag_idx_1], graph[dag_idx_3]) + self.add_control_dependency(graph[dag_idx_1], graph[dag_idx_2]) + self.add_control_dependency(graph[dag_idx_2], graph[dag_idx_3]) + + actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) + assert len(actor_to_execution_schedule) == 1 + assert len(actor_to_execution_schedule[fake_actor]) == 9 + assert actor_to_execution_schedule[fake_actor] == [ + graph[dag_idx_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_3][_DAGNodeOperationType.READ].operation, + graph[dag_idx_3][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_3][_DAGNodeOperationType.WRITE].operation, + ] + + def test_two_actors_no_nccl(self, monkeypatch): + """ + driver -> actor_1.op (dag_idx_1_1) -> actor_2.op (dag_idx_2_2) -> driver + | | + -> actor_2.op (dag_idx_2_1) -> actor_1.op (dag_idx_1_2) - + + Test the case where there are two actors and no NCCL operations. + Because there is no NCCL operation, all operations with smaller + `bind_index` should be executed before the operations with larger + `bind_index` on the same actor. + """ + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + + fake_actor_1 = ActorHandle("fake_actor_1") + dag_idx_1_1, local_idx_1_1 = 1, 0 + dag_idx_1_2, local_idx_1_2 = 4, 1 + + fake_actor_2 = ActorHandle("fake_actor_2") + dag_idx_2_1, local_idx_2_1 = 2, 0 + dag_idx_2_2, local_idx_2_2 = 3, 1 + + graph = { + dag_idx_1_1: generate_dag_graph_nodes( + local_idx_1_1, dag_idx_1_1, fake_actor_1, False + ), + dag_idx_2_1: generate_dag_graph_nodes( + local_idx_2_1, dag_idx_2_1, fake_actor_2, False + ), + dag_idx_2_2: generate_dag_graph_nodes( + local_idx_2_2, dag_idx_2_2, fake_actor_2, False + ), + dag_idx_1_2: generate_dag_graph_nodes( + local_idx_1_2, dag_idx_1_2, fake_actor_1, False + ), + } + self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) + self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_2]) + self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_1_2]) + self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) + self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) + + actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) + assert len(actor_to_execution_schedule) == 2 + assert len(actor_to_execution_schedule[fake_actor_1]) == 6 + assert len(actor_to_execution_schedule[fake_actor_2]) == 6 + + assert actor_to_execution_schedule[fake_actor_1] == [ + graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, + ] + assert actor_to_execution_schedule[fake_actor_2] == [ + graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, + ] + + def test_two_actors_with_nccl(self, monkeypatch): + """ + driver -> actor_1.op (dag_idx_1_1) -> actor_2.op (dag_idx_2_2) -> driver + | | + -> actor_2.op (dag_idx_2_1) -> actor_1.op (dag_idx_1_2) - + + In this test, the communication between fake_actor_1 and fake_actor_2 is done + using NCCL. When the dag_idx_1.WRITE operation is picked, the dag_idx_2.READ + operation is also added to the execution schedule because of the NCCL operation. + """ + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + + fake_actor_1 = ActorHandle("fake_actor_1") + dag_idx_1_1, local_idx_1_1 = 1, 0 + dag_idx_1_2, local_idx_1_2 = 4, 1 + + fake_actor_2 = ActorHandle("fake_actor_2") + dag_idx_2_1, local_idx_2_1 = 2, 0 + dag_idx_2_2, local_idx_2_2 = 3, 1 + + graph = { + dag_idx_1_1: generate_dag_graph_nodes( + local_idx_1_1, dag_idx_1_1, fake_actor_1, True + ), + dag_idx_2_1: generate_dag_graph_nodes( + local_idx_2_1, dag_idx_2_1, fake_actor_2, True + ), + dag_idx_2_2: generate_dag_graph_nodes( + local_idx_2_2, dag_idx_2_2, fake_actor_2, False + ), + dag_idx_1_2: generate_dag_graph_nodes( + local_idx_1_2, dag_idx_1_2, fake_actor_1, False + ), + } + self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) + self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_2]) + self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_1_2]) + self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) + self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) + + actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) + assert len(actor_to_execution_schedule) == 2 + assert len(actor_to_execution_schedule[fake_actor_1]) == 6 + assert len(actor_to_execution_schedule[fake_actor_2]) == 6 + + assert actor_to_execution_schedule[fake_actor_1] == [ + graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, + ] + assert actor_to_execution_schedule[fake_actor_2] == [ + graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + # The order of `dag_idx_2_2.READ` and `dag_idx_2_2.COMPUTE` is important. + graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, + ] + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From c98362becce7f73499bc140b547f80731760d9cd Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Sat, 10 Aug 2024 00:26:32 +0000 Subject: [PATCH 099/111] add a test for 1f1b Signed-off-by: Kai-Hsun Chen --- .../experimental/test_execution_schedule.py | 104 ++++++++++++++++++ 1 file changed, 104 insertions(+) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 0c2b0f6789e7c..5ff8ccd8f1085 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -780,6 +780,110 @@ def test_two_actors_with_nccl(self, monkeypatch): graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, ] + def test_simulate_pp_2workers_2batches_1f1b(self, monkeypatch): + """ + This test simulates a simple 1F1B pipeline parallelism for training with + 2 workers and 2 batches. + + w1: fwd_b1 fwd_b2 bwd_b1 bwd_b2 + w2: fwd_b1 bwd_b1 fwd_b2 bwd_b2 + + The communication between workers is done using NCCL. The communication + within the worker actor is done using IntraProcessChannel. + """ + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + + worker_1 = ActorHandle("worker_1") + dag_idx_1_1, local_idx_1_1 = 1, 0 + dag_idx_1_2, local_idx_1_2 = 2, 1 + dag_idx_1_3, local_idx_1_3 = 3, 2 + dag_idx_1_4, local_idx_1_4 = 4, 3 + worker_2 = ActorHandle("worker_2") + dag_idx_2_1, local_idx_2_1 = 5, 0 + dag_idx_2_2, local_idx_2_2 = 6, 1 + dag_idx_2_3, local_idx_2_3 = 7, 2 + dag_idx_2_4, local_idx_2_4 = 8, 3 + graph = { + dag_idx_1_1: generate_dag_graph_nodes( + local_idx_1_1, dag_idx_1_1, worker_1, True + ), + dag_idx_1_2: generate_dag_graph_nodes( + local_idx_1_2, dag_idx_1_2, worker_1, True + ), + dag_idx_1_3: generate_dag_graph_nodes( + local_idx_1_3, dag_idx_1_3, worker_1, False + ), + dag_idx_1_4: generate_dag_graph_nodes( + local_idx_1_4, dag_idx_1_4, worker_1, False + ), + dag_idx_2_1: generate_dag_graph_nodes( + local_idx_2_1, dag_idx_2_1, worker_2, False + ), + dag_idx_2_2: generate_dag_graph_nodes( + local_idx_2_2, dag_idx_2_2, worker_2, True + ), + dag_idx_2_3: generate_dag_graph_nodes( + local_idx_2_3, dag_idx_2_3, worker_2, False + ), + dag_idx_2_4: generate_dag_graph_nodes( + local_idx_2_4, dag_idx_2_4, worker_2, True + ), + } + self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_3]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_4]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_3]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_4]) + self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_1]) + self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_2_2]) + self.add_data_dependeny(graph[dag_idx_2_2], graph[dag_idx_1_3]) + self.add_data_dependeny(graph[dag_idx_1_2], graph[dag_idx_2_3]) + self.add_data_dependeny(graph[dag_idx_2_3], graph[dag_idx_2_4]) + self.add_data_dependeny(graph[dag_idx_2_4], graph[dag_idx_1_4]) + self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) + self.add_control_dependency(graph[dag_idx_1_2], graph[dag_idx_1_3]) + self.add_control_dependency(graph[dag_idx_1_3], graph[dag_idx_1_4]) + self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) + self.add_control_dependency(graph[dag_idx_2_2], graph[dag_idx_2_3]) + self.add_control_dependency(graph[dag_idx_2_3], graph[dag_idx_2_4]) + + actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) + assert len(actor_to_execution_schedule) == 2 + assert len(actor_to_execution_schedule[worker_1]) == 12 + assert len(actor_to_execution_schedule[worker_2]) == 12 + assert actor_to_execution_schedule[worker_1] == [ + graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_3][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_3][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_3][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_4][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_4][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_4][_DAGNodeOperationType.WRITE].operation, + ] + assert actor_to_execution_schedule[worker_2] == [ + graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + # The order of `dag_idx_2_3.READ` and `dag_idx_2_2.WRITE` is important. + graph[dag_idx_2_3][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_3][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_3][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_4][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_4][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_4][_DAGNodeOperationType.WRITE].operation, + ] + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): From c45298bcb510d7814e8de3d667972723ef9cfe29 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Sat, 10 Aug 2024 00:37:55 +0000 Subject: [PATCH 100/111] add a test for 1f1b without nccl Signed-off-by: Kai-Hsun Chen --- .../experimental/test_execution_schedule.py | 110 +++++++++++++++++- 1 file changed, 109 insertions(+), 1 deletion(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 5ff8ccd8f1085..7fbee2560cd7a 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -780,7 +780,7 @@ def test_two_actors_with_nccl(self, monkeypatch): graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, ] - def test_simulate_pp_2workers_2batches_1f1b(self, monkeypatch): + def test_simulate_pp_2workers_2batches_1f1b_with_nccl(self, monkeypatch): """ This test simulates a simple 1F1B pipeline parallelism for training with 2 workers and 2 batches. @@ -884,6 +884,114 @@ def test_simulate_pp_2workers_2batches_1f1b(self, monkeypatch): graph[dag_idx_2_4][_DAGNodeOperationType.WRITE].operation, ] + def test_simulate_pp_2workers_2batches_1f1b_no_nccl(self, monkeypatch): + """ + This test simulates a simple 1F1B pipeline parallelism for training with + 2 workers and 2 batches. + + w1: fwd_b1 fwd_b2 bwd_b1 bwd_b2 + w2: fwd_b1 bwd_b1 fwd_b2 bwd_b2 + + Because there is no NCCL operation, all operations with smaller + `bind_index` should be executed before the operations with larger + `bind_index` on the same actor. + """ + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + + worker_1 = ActorHandle("worker_1") + dag_idx_1_1, local_idx_1_1 = 1, 0 + dag_idx_1_2, local_idx_1_2 = 2, 1 + dag_idx_1_3, local_idx_1_3 = 3, 2 + dag_idx_1_4, local_idx_1_4 = 4, 3 + worker_2 = ActorHandle("worker_2") + dag_idx_2_1, local_idx_2_1 = 5, 0 + dag_idx_2_2, local_idx_2_2 = 6, 1 + dag_idx_2_3, local_idx_2_3 = 7, 2 + dag_idx_2_4, local_idx_2_4 = 8, 3 + + # No NCCL operation. + graph = { + dag_idx_1_1: generate_dag_graph_nodes( + local_idx_1_1, dag_idx_1_1, worker_1, False + ), + dag_idx_1_2: generate_dag_graph_nodes( + local_idx_1_2, dag_idx_1_2, worker_1, False + ), + dag_idx_1_3: generate_dag_graph_nodes( + local_idx_1_3, dag_idx_1_3, worker_1, False + ), + dag_idx_1_4: generate_dag_graph_nodes( + local_idx_1_4, dag_idx_1_4, worker_1, False + ), + dag_idx_2_1: generate_dag_graph_nodes( + local_idx_2_1, dag_idx_2_1, worker_2, False + ), + dag_idx_2_2: generate_dag_graph_nodes( + local_idx_2_2, dag_idx_2_2, worker_2, False + ), + dag_idx_2_3: generate_dag_graph_nodes( + local_idx_2_3, dag_idx_2_3, worker_2, False + ), + dag_idx_2_4: generate_dag_graph_nodes( + local_idx_2_4, dag_idx_2_4, worker_2, False + ), + } + self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_3]) + self.add_edge_between_read_compute_write(graph[dag_idx_1_4]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_3]) + self.add_edge_between_read_compute_write(graph[dag_idx_2_4]) + self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_1]) + self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_2_2]) + self.add_data_dependeny(graph[dag_idx_2_2], graph[dag_idx_1_3]) + self.add_data_dependeny(graph[dag_idx_1_2], graph[dag_idx_2_3]) + self.add_data_dependeny(graph[dag_idx_2_3], graph[dag_idx_2_4]) + self.add_data_dependeny(graph[dag_idx_2_4], graph[dag_idx_1_4]) + self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) + self.add_control_dependency(graph[dag_idx_1_2], graph[dag_idx_1_3]) + self.add_control_dependency(graph[dag_idx_1_3], graph[dag_idx_1_4]) + self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) + self.add_control_dependency(graph[dag_idx_2_2], graph[dag_idx_2_3]) + self.add_control_dependency(graph[dag_idx_2_3], graph[dag_idx_2_4]) + + actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) + assert len(actor_to_execution_schedule) == 2 + assert len(actor_to_execution_schedule[worker_1]) == 12 + assert len(actor_to_execution_schedule[worker_2]) == 12 + assert actor_to_execution_schedule[worker_1] == [ + graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_3][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_3][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_3][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_1_4][_DAGNodeOperationType.READ].operation, + graph[dag_idx_1_4][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_1_4][_DAGNodeOperationType.WRITE].operation, + ] + assert actor_to_execution_schedule[worker_2] == [ + graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + # The order of `dag_idx_2_3.READ` and `dag_idx_2_2.WRITE` is important. + # It is different from the case where there is an NCCL operation. + graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_3][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_3][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_3][_DAGNodeOperationType.WRITE].operation, + graph[dag_idx_2_4][_DAGNodeOperationType.READ].operation, + graph[dag_idx_2_4][_DAGNodeOperationType.COMPUTE].operation, + graph[dag_idx_2_4][_DAGNodeOperationType.WRITE].operation, + ] + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): From e006cd8c8fed701a4da6ed38c353393147998cb6 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Sat, 10 Aug 2024 02:46:39 +0000 Subject: [PATCH 101/111] simplify 1f1b test --- python/ray/dag/BUILD | 2 +- .../test_execution_schedule_gpu.py | 287 +++++------------- 2 files changed, 70 insertions(+), 219 deletions(-) diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 9dd5f9aec1bb4..02ca1b4088807 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -149,7 +149,7 @@ py_test( ) py_test( - name = "test_execution_schedule", + name = "test_execution_schedule_gpu", size = "medium", srcs = [ "tests/experimental/test_execution_schedule_gpu.py", diff --git a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py index 2b97832f19885..294e394538fe5 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py @@ -11,10 +11,8 @@ from ray.dag import InputNode, MultiOutputNode from ray.dag.dag_node_operation import _DAGNodeOperationType import torch -from typing import List -from dataclasses import dataclass, field -from collections import deque, defaultdict -from ray.actor import ActorHandle +from typing import Optional +from ray.dag.compiled_dag_node import CompiledDAG if sys.platform != "linux" and sys.platform != "darwin": pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) @@ -22,220 +20,25 @@ USE_GPU = bool(os.environ.get("RAY_PYTEST_USE_GPU", 0)) -@dataclass -class PipelineConfig: - """ - pp_size: Number of pipeline parallel workers. - num_micro_batches: Number of micro-batches. - """ - - pp_size: int - num_micro_batches: int - - -@dataclass -class PipelineUnit: - """ - op: Operation type (FWD or BWD). - pp_rank: Pipeline parallel rank. - batch_id: Batch ID. - uid: Unique ID for the pipeline unit. - """ - - op: str - pp_rank: int - batch_id: int - uid: str = field(init=False, repr=False) - - def __post_init__(self): - self.uid = f"{self.op}_rank-{self.pp_rank}_batch-{self.batch_id}" - - def __repr__(self) -> str: - return self.uid - - -def generate_1f1b_schedule(config: PipelineConfig) -> List[List[PipelineUnit]]: - """ - Args: - config: Pipeline configuration. - Returns: - schedule: List of pipeline units for 1F1B pipeline parallelism. Each - inner list represents the schedule for a pipeline parallel worker. - """ - pp_size = config.pp_size - num_micro_batches = config.num_micro_batches - - schedule = [] - for pp_rank in range(config.pp_size): - warm_up_batches = pp_size - pp_rank - main_1f1b_batches = num_micro_batches - warm_up_batches - cool_down_batches = num_micro_batches - main_1f1b_batches - - rank_schedule = [] - bwd_batch_id = fwd_batch_id = 0 - - for _ in range(warm_up_batches): - rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) - fwd_batch_id += 1 - - for _ in range(main_1f1b_batches): - rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) - bwd_batch_id += 1 - rank_schedule.append(PipelineUnit("FWD", pp_rank, fwd_batch_id)) - fwd_batch_id += 1 - - for _ in range(cool_down_batches): - rank_schedule.append(PipelineUnit("BWD", pp_rank, bwd_batch_id)) - bwd_batch_id += 1 - schedule.append(rank_schedule) - return schedule - - -class PipelineModel: - def __init__( - self, - config: PipelineConfig, - schedule: List[List[PipelineUnit]], - blocks: List[ActorHandle], - ) -> None: - """ - Args: - config: Pipeline configuration. - schedule: List of pipeline units. Each inner list represents the - schedule for a pipeline parallel worker. - blocks: List of actors representing pipeline parallel workers. - """ - self.config = config - self.blocks = blocks - self.generate_pipeline_schedules(schedule) - self.dag = self.build_dag() - - def generate_pipeline_schedules(self, schedule: List[List[PipelineUnit]]): - """ - Convert per-worker schedule to per-batch schedule. - - Args: - schedule: List of pipeline units. Each inner list represents the - schedule for a pipeline parallel worker. - """ - self.id_to_unit = dict() - self.stage_schedules = defaultdict(list) - self.batch_schedules = defaultdict(list) - - for pp_rank, stage_schedule in enumerate(schedule): - self.stage_schedules[pp_rank] = stage_schedule - for unit in stage_schedule: - self.id_to_unit[unit.uid] = unit - self.batch_schedules[unit.batch_id].append(unit) - - for batch_id in self.batch_schedules: - fwd_units = [ - unit for unit in self.batch_schedules[batch_id] if unit.op == "FWD" - ] - bwd_units = [ - unit for unit in self.batch_schedules[batch_id] if unit.op == "BWD" - ] - - fwd_units.sort(key=lambda unit: unit.pp_rank) - bwd_units.sort(key=lambda unit: unit.pp_rank, reverse=True) - self.batch_schedules[batch_id] = fwd_units + bwd_units - - def build_dependency_graph(self): - """ - Add dependencies between pipeline units based on: - (1) Per-batch schedule and (2) Per-worker schedule. - """ - graph = defaultdict(set) - reversed_graph = defaultdict(set) - - for schedules in [self.batch_schedules, self.stage_schedules]: - for schedule in schedules.values(): - prev_unit = None - for unit in schedule: - if prev_unit: - graph[prev_unit.uid].add(unit.uid) - reversed_graph[unit.uid].add(prev_unit.uid) - prev_unit = unit - return graph, reversed_graph - - def build_dag(self): - """ - Build accelerated DAG for the pipeline model. - """ - graph, reversed_graph = self.build_dependency_graph() - dag_nodes = dict() # Cache DAG Node for each unit - - first_unit = self.batch_schedules[0][0] - queue = deque([first_unit.uid]) - - with InputNode() as input_node: - root_node = self.blocks[0].read_input.bind(input_node) - - output_nodes = [] - - while queue: - uid = queue.popleft() - unit = self.id_to_unit[uid] - batch_id = unit.batch_id - batch_schedule_index = self.batch_schedules[batch_id].index(unit) - - # First forward step - if batch_schedule_index == 0: - prev_dag_node = root_node - else: - prev_unit = self.batch_schedules[batch_id][batch_schedule_index - 1] - prev_dag_node = dag_nodes[prev_unit.uid] - - block = self.blocks[unit.pp_rank] - if unit.op == "FWD": - cur_dag_node = block.fwd.bind(prev_dag_node) - else: - cur_dag_node = block.bwd.bind(prev_dag_node) - - # Last backward step - if batch_schedule_index == 2 * self.config.pp_size - 1: - output_nodes.append(cur_dag_node) - - # ADD NCCL Channel: - if unit.op == "FWD" and unit.pp_rank < self.config.pp_size - 1: - cur_dag_node.with_type_hint( - TorchTensorType(transport=TorchTensorType.NCCL) - ) - if unit.op == "BWD" and unit.pp_rank > 0: - cur_dag_node.with_type_hint( - TorchTensorType(transport=TorchTensorType.NCCL) - ) - - dag_nodes[uid] = cur_dag_node - - # Enqueue new units - for target_uid in graph[uid]: - reversed_graph[target_uid].remove(uid) - if not reversed_graph[target_uid]: - queue.append(target_uid) - - dag = MultiOutputNode(output_nodes) - compiled_dag = dag.experimental_compile() - return compiled_dag - - def step(self, input_batches): - return ray.get(self.dag.execute(input_batches)) - - def teardown(self): - self.dag.teardown() - - @ray.remote(num_cpus=0, num_gpus=1) class Worker: - def __init__(self): - pass + def __init__(self, rank: Optional[int] = None): + self.rank = rank + self.trace = [] def fwd(self, value): + self.trace.append(("FWD", self.rank)) return value def bwd(self, value): + self.trace.append(("BWD", self.rank)) return value + def pop_trace(self): + trace = self.trace + self.trace = [] + return trace + def read_input(self, input): return input @@ -246,6 +49,56 @@ def no_op_two(self, value1, value2): return value1, value2 +def generate_1f1b_dag( + num_workers: int, num_microbatches: int, num_lead_microbatches: int +) -> CompiledDAG: + workers = [Worker.remote(rank) for rank in range(num_workers)] + + with ray.dag.InputNode() as inp: + fwd_queues = [[] for _ in range(num_workers)] + bwd_queues = [[] for _ in range(num_workers)] + # Once a worker's counter reaches 0, it cannot execute another fwd until it + # executes a bwd first. + fwd_counter = [num_lead_microbatches - i for i in range(num_workers)] + # All of the done batches. + done = [] + + # FWD on worker 0. + input_data = workers[0].read_input.bind(inp) + for i in range(num_microbatches): + fwd_queues[0].append(input_data) + + while len(done) < num_microbatches: + for i, worker in enumerate(workers): + if fwd_counter[i] > 0 and fwd_queues[i]: + b = fwd_queues[i].pop(0) + b = worker.fwd.bind(b) + if i < num_workers - 1: + fwd_queues[i + 1].append(b) + # Use NCCL channel for communication between workers. + b.with_type_hint( + TorchTensorType(transport=TorchTensorType.NCCL) + ) + else: + bwd_queues[i].append(b) + fwd_counter[i] -= 1 + elif bwd_queues[i]: + b = bwd_queues[i].pop(0) + b = worker.bwd.bind(b) + if i > 0: + bwd_queues[i - 1].append(b) + # Use NCCL channel for communication between workers. + b.with_type_hint( + TorchTensorType(transport=TorchTensorType.NCCL) + ) + else: + done.append(b) + fwd_counter[i] += 1 + dag = ray.dag.MultiOutputNode(done) + compiled_dag = dag.experimental_compile() + return compiled_dag + + @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 2}], indirect=True) def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): """ @@ -353,20 +206,18 @@ def test_simulate_pp_4workers_8batches_1f1b(ray_start_regular, monkeypatch): monkeypatch.setattr(ray.dag.constants, "RAY_ADAG_ENABLE_DETECT_DEADLOCK", False) - num_worker, num_batch = 4, 8 - - workers = [Worker.remote() for _ in range(num_worker)] - config = PipelineConfig(num_worker, num_batch) - schedule = generate_1f1b_schedule(config) - model = PipelineModel(config, schedule, workers) + num_workers, num_microbatches, num_lead_microbatches = 4, 8, 4 + compiled_dag = generate_1f1b_dag( + num_workers, num_microbatches, num_lead_microbatches + ) tensor_cpu = torch.zeros(10, 10) - tensors = model.step(tensor_cpu) + tensors = ray.get(compiled_dag.execute(tensor_cpu)) tensor_cuda = tensor_cpu.to("cuda:0") - assert len(tensors) == num_batch + assert len(tensors) == num_microbatches for t in tensors: assert torch.equal(t, tensor_cuda) - model.teardown() + compiled_dag.teardown() @pytest.mark.parametrize("ray_start_regular", [{"num_gpus": 3}], indirect=True) From 5b7c31898d5306550a1231693654487e43633554 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Sat, 10 Aug 2024 02:56:49 +0000 Subject: [PATCH 102/111] troubleshoot ci --- .buildkite/core.rayci.yml | 2 - .../test_execution_schedule_gpu.py | 41 +++++++++++++++++++ 2 files changed, 41 insertions(+), 2 deletions(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 8caa6b9e78684..d5a86da793087 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -376,8 +376,6 @@ steps: - gpu instance_type: gpu-large commands: - # This machine has 4 GPUs, and we need 2 GPUs, so allow 2 tests to run in - # parallel. - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core --build-name coregpubuild --only-tags multi_gpu diff --git a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py index 294e394538fe5..c3aa6fdd6e23c 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py @@ -13,6 +13,7 @@ import torch from typing import Optional from ray.dag.compiled_dag_node import CompiledDAG +from ray.dag.tests.experimental.test_torch_tensor_dag import TorchTensorWorker if sys.platform != "linux" and sys.platform != "darwin": pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) @@ -366,6 +367,46 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): compiled_dag.teardown() +@pytest.mark.parametrize("ray_start_regular", [{"num_cpus": 4}], indirect=True) +def test_torch_tensor_nccl_nested_dynamic(ray_start_regular): + """ + Test nested torch.Tensor passed via NCCL. Its shape and dtype is + dynamically declared, and there may be multiple tensors. + """ + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + assert ( + sum(node["Resources"].get("GPU", 0) for node in ray.nodes()) > 1 + ), "This test requires at least 2 GPUs" + + actor_cls = TorchTensorWorker.options(num_gpus=1) + + sender = actor_cls.remote() + receiver = actor_cls.remote() + + with InputNode() as inp: + dag = sender.send_dict_with_tuple_args.bind(inp) + dag = dag.with_type_hint(TorchTensorType(transport="nccl")) + dag = receiver.recv_dict.bind(dag) + + compiled_dag = dag.experimental_compile() + + for i in range(3): + i += 1 + + shape = (10 * i,) + dtype = torch.float16 + args = (shape, dtype, i) + + ref = compiled_dag.execute(args) + result = ray.get(ref) + expected_result = {j: (j, shape, dtype) for j in range(i)} + assert result == expected_result + + compiled_dag.teardown() + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From 5485c3098a5d2d9a34754159cd331658bf4a259b Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Sat, 10 Aug 2024 04:35:44 +0000 Subject: [PATCH 103/111] troubleshoot ci Signed-off-by: Kai-Hsun Chen --- .buildkite/core.rayci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index d5a86da793087..8dc9d036e47d8 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -379,4 +379,5 @@ steps: - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core --build-name coregpubuild --only-tags multi_gpu + - sleep 15000 depends_on: coregpubuild From 60b164352d911a3f2d4152595d2b2bbe5a9884f0 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Sat, 10 Aug 2024 06:16:56 +0000 Subject: [PATCH 104/111] troubleshoot ci Signed-off-by: Kai-Hsun Chen --- .buildkite/core.rayci.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 8dc9d036e47d8..af9a7811128f1 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -376,8 +376,8 @@ steps: - gpu instance_type: gpu-large commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core - --build-name coregpubuild - --only-tags multi_gpu - - sleep 15000 + - sleep 10000 + # - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core + # --build-name coregpubuild + # --only-tags multi_gpu depends_on: coregpubuild From 28d87604572b5e4c0829be2e7b94d2ede3bea407 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 12 Aug 2024 19:13:37 +0000 Subject: [PATCH 105/111] revert troubleshoot ci Signed-off-by: Kai-Hsun Chen --- .buildkite/core.rayci.yml | 7 ++-- python/ray/dag/BUILD | 18 -------- .../test_execution_schedule_gpu.py | 41 ------------------- 3 files changed, 3 insertions(+), 63 deletions(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index af9a7811128f1..d5a86da793087 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -376,8 +376,7 @@ steps: - gpu instance_type: gpu-large commands: - - sleep 10000 - # - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core - # --build-name coregpubuild - # --only-tags multi_gpu + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core + --build-name coregpubuild + --only-tags multi_gpu depends_on: coregpubuild diff --git a/python/ray/dag/BUILD b/python/ray/dag/BUILD index 02ca1b4088807..72c8b4b38b22f 100644 --- a/python/ray/dag/BUILD +++ b/python/ray/dag/BUILD @@ -147,21 +147,3 @@ py_test( ], deps = ["//:ray_lib"], ) - -py_test( - name = "test_execution_schedule_gpu", - size = "medium", - srcs = [ - "tests/experimental/test_execution_schedule_gpu.py", - ], - env = {"RAY_PYTEST_USE_GPU": "1"}, - main = "tests/experimental/test_execution_schedule_gpu.py", - tags = [ - "accelerated_dag", - "exclusive", - "multi_gpu", - "no_windows", - "team:core", - ], - deps = ["//:ray_lib"], -) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py index c3aa6fdd6e23c..294e394538fe5 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py @@ -13,7 +13,6 @@ import torch from typing import Optional from ray.dag.compiled_dag_node import CompiledDAG -from ray.dag.tests.experimental.test_torch_tensor_dag import TorchTensorWorker if sys.platform != "linux" and sys.platform != "darwin": pytest.skip("Skipping, requires Linux or Mac.", allow_module_level=True) @@ -367,46 +366,6 @@ def test_three_actors_with_nccl_2(ray_start_regular, monkeypatch): compiled_dag.teardown() -@pytest.mark.parametrize("ray_start_regular", [{"num_cpus": 4}], indirect=True) -def test_torch_tensor_nccl_nested_dynamic(ray_start_regular): - """ - Test nested torch.Tensor passed via NCCL. Its shape and dtype is - dynamically declared, and there may be multiple tensors. - """ - if not USE_GPU: - pytest.skip("NCCL tests require GPUs") - - assert ( - sum(node["Resources"].get("GPU", 0) for node in ray.nodes()) > 1 - ), "This test requires at least 2 GPUs" - - actor_cls = TorchTensorWorker.options(num_gpus=1) - - sender = actor_cls.remote() - receiver = actor_cls.remote() - - with InputNode() as inp: - dag = sender.send_dict_with_tuple_args.bind(inp) - dag = dag.with_type_hint(TorchTensorType(transport="nccl")) - dag = receiver.recv_dict.bind(dag) - - compiled_dag = dag.experimental_compile() - - for i in range(3): - i += 1 - - shape = (10 * i,) - dtype = torch.float16 - args = (shape, dtype, i) - - ref = compiled_dag.execute(args) - result = ray.get(ref) - expected_result = {j: (j, shape, dtype) for j in range(i)} - assert result == expected_result - - compiled_dag.teardown() - - if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From a2720374d7f23642f872204dedfd1b2fc67f7877 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 12 Aug 2024 19:14:45 +0000 Subject: [PATCH 106/111] revert troubleshoot ci Signed-off-by: Kai-Hsun Chen --- .buildkite/core.rayci.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index d5a86da793087..2929afca4a44d 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -376,7 +376,10 @@ steps: - gpu instance_type: gpu-large commands: + # This machine has 4 GPUs, and we need 2 GPUs, so allow 2 tests to run in + # parallel. - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... core + --parallelism-per-worker 2 --gpus 2 --build-name coregpubuild --only-tags multi_gpu depends_on: coregpubuild From 1d9894bcd338e3eb5f9853626dc08324ca79e94b Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 13 Aug 2024 09:29:07 -0700 Subject: [PATCH 107/111] Update python/ray/dag/dag_node_operation.py Co-authored-by: Stephanie Wang Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 1baa2e883bcfd..78f2ce12d1ba0 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -29,7 +29,7 @@ def __init__( Args: local_idx: The index of the task that this operation belongs to in the actor's ExecutableTask list. The index is not the same - as bind_index, but there are positive correlations between the two. + as bind_index because there may be more tasks bound to an actor than tasks that appear in the current compiled DAG. operation_type: The type of operation to perform. """ self.local_idx = local_idx From c3819e418b5bda04a3177f4a465a2226d079b8ca Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Tue, 13 Aug 2024 16:53:47 +0000 Subject: [PATCH 108/111] update comments for _select_next_nodes Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 78f2ce12d1ba0..19b4d80b4b46b 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -29,7 +29,8 @@ def __init__( Args: local_idx: The index of the task that this operation belongs to in the actor's ExecutableTask list. The index is not the same - as bind_index because there may be more tasks bound to an actor than tasks that appear in the current compiled DAG. + as bind_index because there may be more tasks bound to an actor + than tasks that appear in the current compiled DAG. operation_type: The type of operation to perform. """ self.local_idx = local_idx @@ -132,7 +133,9 @@ def _select_next_nodes( This function may return multiple nodes if they are NCCL nodes. In that case, this function only removes the NCCL write node, which is also the head of a - priority queue. Other nodes will be removed in the following iterations. + priority queue. Other nodes will be removed in the following iterations. The + NCCL read nodes will be returned even though they should not yet be in the + candidate list. Args: actor_to_candidates: A dictionary mapping an actor id to a list of From 3e5feed6168843d699da4bd59e6f5b36d80ae19c Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 14 Aug 2024 06:23:39 +0000 Subject: [PATCH 109/111] change __lt__ for _DAGOperationGraphNode Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 108 ++++++++++-------- .../experimental/test_execution_schedule.py | 5 +- .../test_execution_schedule_gpu.py | 4 +- 3 files changed, 68 insertions(+), 49 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index 19b4d80b4b46b..ea45165447644 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -1,6 +1,6 @@ from functools import total_ordering from enum import Enum -from typing import Set, Tuple, List, Dict, Optional +from typing import Set, Tuple, List, Dict import ray import heapq from collections import defaultdict @@ -76,22 +76,40 @@ def in_degree(self) -> int: def __lt__(self, other: "_DAGOperationGraphNode"): """ - Two _DAGOperationGraphNodes are comparable only when they belong to - the same actor. For operations on the same actor, if `local_idx` is smaller, - the DAGNode to which this operation belongs has a smaller `bind_index`. + This function defines the order of the nodes in the priority queue used in + `_select_next_nodes`. The priority queue is a min-heap, so the node with + higher priority is considered "less than" the other node. """ - assert self.actor_handle == other.actor_handle - return self.operation.local_idx < other.operation.local_idx + # If two nodes belong to the same actor, select the one with + # the smaller `local_idx`. + if self.actor_handle == other.actor_handle: + return self.operation.local_idx < other.operation.local_idx + # If two nodes belong to different actors and one of them is an NCCL + # write node, select the one that is not an NCCL write node. + is_nccl_write = ( + self.operation.type == _DAGNodeOperationType.WRITE and self.requires_nccl + ) + other_is_nccl_write = ( + other.operation.type == _DAGNodeOperationType.WRITE and other.requires_nccl + ) + if is_nccl_write != other_is_nccl_write: + return not is_nccl_write + # If two nodes belong to different actors and both are either NCCL write + # nodes or neither are NCCL write nodes, select the one with the smaller + # `local_idx`. If they have the same `local_idx`, select the one with the + # smaller `dag_idx`. + if self.operation.local_idx != other.operation.local_idx: + return self.operation.local_idx < other.operation.local_idx + return self.dag_idx < other.dag_idx def __eq__(self, other: "_DAGOperationGraphNode"): """ - Two _DAGOperationGraphNodes are comparable only when they belong to the - same actor. For operations on the same actor, two operations are equal - only when they have the same `local_idx` and `type`. + Two operations are equal only when they have the same `local_idx` and `type` + and belong to the same actor. """ - assert self.actor_handle == other.actor_handle return ( - self.operation.local_idx == other.operation.local_idx + self.actor_handle == other.actor_handle + and self.operation.local_idx == other.operation.local_idx and self.operation.type == other.operation.type ) @@ -117,19 +135,26 @@ def _select_next_nodes( ): """ This function selects the next nodes for topological sort to generate execution - schedule. If there are multiple candidate _DAGOperationGraphNodes, select nodes - based on the following rules: - - #1 If the nodes are not NCCL write nodes, select the one with the smallest - `bind_index`. If there are multiple candidate nodes with the smallest - `bind_index` among the actors to which they belong, any one of them is - acceptable, but the implementation ensures the result is deterministic. - For the implementation details, we maintain a priority queue for each actor, - where the head of the priority queue is the node with the smallest `bind_index`. - #2 If #1 cannot be satisfied, it means that all candidate nodes are NCCL write - nodes. In this case, select the one at the head of the priority queue and - its immediately downstream nodes, which are NCCL read nodes, regardless of - whether the downstream nodes are heads of their own priority queues. + schedule. If there are multiple candidate _DAGOperationGraphNodes, select the node + with the top priority based on the following rules: + + #1 If two candidate nodes belong to the same actor, select the one with + the smaller `local_idx`. + + #2 If two candidate nodes belong to different actors and both are either NCCL + write nodes or neither are NCCL write nodes, select the one with the smaller + `local_idx`. If they have the same `local_idx`, select the one with the + smaller `dag_idx`. + + #3 If two candidate nodes belong to different actors and one of them is an NCCL + write node, select the one that is not an NCCL write node. + + For the implementation details, we maintain a priority queue for each actor, + where the head of the priority queue is the node with the smallest `local_idx`. + + If the selected node is an NCCL write node, select all its immediately downstream + nodes, which are NCCL read nodes, regardless of whether the downstream nodes are + heads of their own priority queues. This function may return multiple nodes if they are NCCL nodes. In that case, this function only removes the NCCL write node, which is also the head of a @@ -149,38 +174,31 @@ def _select_next_nodes( A list of _DAGOperationGraphNodes to be placed into the corresponding execution schedules. """ + top_priority_node = None next_nodes: List[_DAGOperationGraphNode] = [] for _, candidates in actor_to_candidates.items(): - if not ( - candidates[0].requires_nccl - and candidates[0].operation.type == _DAGNodeOperationType.WRITE - ): - next_nodes.append(heapq.heappop(candidates)) - assert len(next_nodes) == 1 - return next_nodes - - first_nccl_node: Optional[_DAGOperationGraphNode] = None - for _, candidates in actor_to_candidates.items(): - if ( - candidates[0].requires_nccl - and candidates[0].operation.type == _DAGNodeOperationType.WRITE - ): - first_nccl_node = candidates[0] - break - - assert first_nccl_node is not None + if top_priority_node is None or candidates[0] < top_priority_node: + top_priority_node = candidates[0] + assert top_priority_node is not None next_nodes.append( - heapq.heappop(actor_to_candidates[first_nccl_node.actor_handle._actor_id]) + heapq.heappop(actor_to_candidates[top_priority_node.actor_handle._actor_id]) ) + if not ( + top_priority_node.operation.type == _DAGNodeOperationType.WRITE + and top_priority_node.requires_nccl + ): + assert len(next_nodes) == 1 + return next_nodes + # An NCCL write node is picked. NCCL is a blocking operation, so we need to pick all # the corresponding NCCL read nodes to avoid a deadlock. - for downstream_node_metadata in first_nccl_node.out_edges: + for downstream_node_metadata in top_priority_node.out_edges: dag_idx, op_type = downstream_node_metadata[0], downstream_node_metadata[1] downstream_node = graph[dag_idx][op_type] assert downstream_node.operation.type == _DAGNodeOperationType.READ next_nodes.append(downstream_node) - assert len(next_nodes) == 1 + len(first_nccl_node.out_edges) + assert len(next_nodes) == 1 + len(top_priority_node.out_edges) return next_nodes diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 7fbee2560cd7a..46d285b63890e 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -52,7 +52,7 @@ class TestSelectNextNodes: local_idx: The DAG node's index in the actor's `executable_tasks` list. """ - def test_two_candidates_on_same_actor(self): + def test_two_candidates_on_same_actor(self, monkeypatch): """ Simulate the case where there are two candidates on the same actor. The candidate with the smaller index in the `executable_tasks` list @@ -65,7 +65,8 @@ def test_two_candidates_on_same_actor(self): list should be selected first; therefore, the one on the left side will be selected first. """ - fake_actor = "fake_actor" + monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) + fake_actor = ActorHandle("fake_actor") # The DAG node has a global index of 1, and its index in the # actor's `executable_tasks` list is 0. dag_idx_1 = 1 diff --git a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py index 294e394538fe5..3b07b5f7615eb 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule_gpu.py @@ -150,8 +150,8 @@ def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): (1, _DAGNodeOperationType.WRITE), (2, _DAGNodeOperationType.READ), (2, _DAGNodeOperationType.COMPUTE), - (2, _DAGNodeOperationType.WRITE), (3, _DAGNodeOperationType.READ), + (2, _DAGNodeOperationType.WRITE), (3, _DAGNodeOperationType.COMPUTE), (3, _DAGNodeOperationType.WRITE), (4, _DAGNodeOperationType.READ), @@ -164,8 +164,8 @@ def test_simulate_pp_2workers_2batches_1f1b(ray_start_regular, monkeypatch): (0, _DAGNodeOperationType.WRITE), (1, _DAGNodeOperationType.READ), (1, _DAGNodeOperationType.COMPUTE), - (2, _DAGNodeOperationType.READ), (1, _DAGNodeOperationType.WRITE), + (2, _DAGNodeOperationType.READ), (2, _DAGNodeOperationType.COMPUTE), (2, _DAGNodeOperationType.WRITE), (3, _DAGNodeOperationType.READ), From d97fda6fd508d4dbc7da893d25ca4b1dd830e20d Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 14 Aug 2024 06:32:36 +0000 Subject: [PATCH 110/111] remove delete key Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index ea45165447644..fa12f35262aa7 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -177,6 +177,8 @@ def _select_next_nodes( top_priority_node = None next_nodes: List[_DAGOperationGraphNode] = [] for _, candidates in actor_to_candidates.items(): + if len(candidates) == 0: + continue if top_priority_node is None or candidates[0] < top_priority_node: top_priority_node = candidates[0] assert top_priority_node is not None @@ -328,8 +330,10 @@ def _generate_actor_to_execution_schedule( visited_nodes = set() - # Topological sort - while actor_to_candidates: + # Use topological sort algorithm to generate the execution schedule. Each iteration + # pops a candidate node from `actor_to_candidates` and each DAG node consists of + # three operations: READ, COMPUTE, and WRITE. + for _ in range(len(graph) * 3): # The function `_select_next_nodes` will pop a candidate node from # `actor_to_candidates` and return a list of nodes that can be executed # in the next step. If multiple nodes are returned, only the NCCL write @@ -348,11 +352,6 @@ def _generate_actor_to_execution_schedule( actor_to_candidates[out_node.actor_handle._actor_id], out_node, ) - - delete_keys = [] - for actor_id, candidates in actor_to_candidates.items(): - if len(candidates) == 0: - delete_keys.append(actor_id) - for key in delete_keys: - del actor_to_candidates[key] + for _, candidates in actor_to_candidates.items(): + assert len(candidates) == 0 return actor_to_execution_schedule From 3241cdaf6bcee89b4563163d8e9ceafd59bc99c8 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 14 Aug 2024 06:38:38 +0000 Subject: [PATCH 111/111] update comments Signed-off-by: Kai-Hsun Chen --- python/ray/dag/dag_node_operation.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/python/ray/dag/dag_node_operation.py b/python/ray/dag/dag_node_operation.py index fa12f35262aa7..7492bc77a7b60 100644 --- a/python/ray/dag/dag_node_operation.py +++ b/python/ray/dag/dag_node_operation.py @@ -154,13 +154,10 @@ def _select_next_nodes( If the selected node is an NCCL write node, select all its immediately downstream nodes, which are NCCL read nodes, regardless of whether the downstream nodes are - heads of their own priority queues. - - This function may return multiple nodes if they are NCCL nodes. In that case, - this function only removes the NCCL write node, which is also the head of a - priority queue. Other nodes will be removed in the following iterations. The - NCCL read nodes will be returned even though they should not yet be in the - candidate list. + heads of their own priority queues. In that case, this function only removes the + NCCL write node, which is also the head of a priority queue. Other nodes will be + removed in the following iterations. The NCCL read nodes will be returned even + though they should not yet be in the candidate list. Args: actor_to_candidates: A dictionary mapping an actor id to a list of