From b134d4d70e27785bbf263f0be00623434302521d Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Tue, 6 Aug 2024 16:11:46 -0500 Subject: [PATCH 01/24] Enable specifying hostname(s) in DragonRunRequests --- doc/changelog.md | 1 + .../_core/launcher/dragon/dragonBackend.py | 216 +++++-- .../_core/launcher/dragon/dragonLauncher.py | 6 +- smartsim/_core/launcher/dragon/pqueue.py | 344 +++++++++++ smartsim/_core/launcher/step/dragonStep.py | 10 +- smartsim/_core/schemas/dragonRequests.py | 10 +- smartsim/settings/dragonRunSettings.py | 8 + tests/dragon/test_worker_manager.py | 2 + tests/test_dragon_run_policy.py | 28 +- tests/test_dragon_run_request.py | 144 ++++- tests/test_dragon_run_request_nowlm.py | 6 +- tests/test_node_prioritizer.py | 550 ++++++++++++++++++ 12 files changed, 1238 insertions(+), 87 deletions(-) create mode 100644 smartsim/_core/launcher/dragon/pqueue.py create mode 100644 tests/test_node_prioritizer.py diff --git a/doc/changelog.md b/doc/changelog.md index 9240efbc8..964e62b49 100644 --- a/doc/changelog.md +++ b/doc/changelog.md @@ -13,6 +13,7 @@ Jump to: Description +- Enable hostname selection for dragon tasks - Remove pydantic dependency from MLI code - Update MLI environment variables using new naming convention - Reduce a copy by using torch.from_numpy instead of torch.tensor diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index daf18e2cb..19e4b6fe6 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -45,6 +45,8 @@ import dragon.native.process_group as dragon_process_group import dragon.native.machine as dragon_machine +from smartsim._core.launcher.dragon.pqueue import NodePrioritizer, PrioritizerFilter + # pylint: enable=import-error # isort: on from ...._core.config import get_config @@ -69,6 +71,10 @@ logger = get_logger(__name__) +# tracking structure for [num_refs, node_name, is_dirty] +_NodeRefCount = t.List[t.Union[int, str, bool]] + + class DragonStatus(str, Enum): ERROR = str(dragon_group_state.Error()) RUNNING = str(dragon_group_state.Running()) @@ -190,41 +196,60 @@ def __init__(self, pid: int) -> None: self._view = DragonBackendView(self) logger.debug(self._view.host_desc) self._infra_ddict: t.Optional[dragon_ddict.DDict] = None + self._prioritizer = NodePrioritizer(self._nodes, self._queue_lock) @property def hosts(self) -> list[str]: with self._queue_lock: return self._hosts + # todo: remove @property def allocated_hosts(self) -> dict[str, str]: + """A map of host names to the step id executing on a host + + :returns: Dictionary with host name as key and step id as value""" with self._queue_lock: return self._allocated_hosts @property - def free_hosts(self) -> t.Deque[str]: + def free_hosts(self) -> t.List[str]: + """Find hosts that do not have a step assigned + + :returns: List of host names""" with self._queue_lock: - return self._free_hosts + return [str(item[1]) for item in self._prioritizer.unassigned()] @property def group_infos(self) -> dict[str, ProcessGroupInfo]: + """Find information pertaining to process groups executing on a host + + :returns: Dictionary with host name as key and group information as value""" with self._queue_lock: return self._group_infos def _initialize_hosts(self) -> None: + """Prepare metadata about the allocation""" with self._queue_lock: self._nodes = [ dragon_machine.Node(node) for node in dragon_machine.System().nodes ] self._hosts: t.List[str] = sorted(node.hostname for node in self._nodes) + """List of hosts available in allocation""" self._cpus = [node.num_cpus for node in self._nodes] + """List of cpu-count by node""" self._gpus = [node.num_gpus for node in self._nodes] - - """List of hosts available in allocation""" - self._free_hosts: t.Deque[str] = collections.deque(self._hosts) - """List of hosts on which steps can be launched""" - self._allocated_hosts: t.Dict[str, str] = {} - """Mapping of hosts on which a step is already running to step ID""" + """List of gpu-count by node""" + # NOTE: self._allocated_hosts does NOT support tracking colocated items + # and it must be converted into a list of step IDs in the future. Now? + self._allocated_hosts: t.Dict[str, str] = {} # todo: rename to _host_tasks? + """Mapping of hosts to an assigned, running step ID""" + self._ref_map: t.Dict[str, _NodeRefCount] = {} + """Map node names to a ref counter for direct access""" + self._cpu_refs: t.List[_NodeRefCount] = [] + """Track reference counts to CPU-only nodes""" + self._gpu_refs: t.List[_NodeRefCount] = [] + """Track reference counts to GPU nodes""" def __str__(self) -> str: return self.status_message @@ -233,7 +258,7 @@ def __str__(self) -> str: def status_message(self) -> str: """Message with status of available nodes and history of launched jobs. - :returns: Status message + :returns: a status message """ return ( "Dragon server backend update\n" @@ -245,9 +270,8 @@ def _heartbeat(self) -> None: @property def cooldown_period(self) -> int: - """Time (in seconds) the server will wait before shutting down - - when exit conditions are met (see ``should_shutdown()`` for further details). + """Time (in seconds) the server will wait before shutting down when + exit conditions are met (see ``should_shutdown()`` for further details). """ return self._cooldown_period @@ -281,6 +305,8 @@ def should_shutdown(self) -> bool: and it requested immediate shutdown, or if it did not request immediate shutdown, but all jobs have been executed. In both cases, a cooldown period may need to be waited before shutdown. + + :returns: `True` if the server should terminate, otherwise `False` """ if self._shutdown_requested and self._can_shutdown: return self._has_cooled_down @@ -288,7 +314,9 @@ def should_shutdown(self) -> bool: @property def current_time(self) -> float: - """Current time for DragonBackend object, in seconds since the Epoch""" + """Current time for DragonBackend object, in seconds since the Epoch + + :returns: the current timestamp""" return time.time() def _can_honor_policy( @@ -296,10 +324,63 @@ def _can_honor_policy( ) -> t.Tuple[bool, t.Optional[str]]: """Check if the policy can be honored with resources available in the allocation. - :param request: DragonRunRequest containing policy information + + :param request: `DragonRunRequest` to validate :returns: Tuple indicating if the policy can be honored and an optional error message""" # ensure the policy can be honored + if request.policy: + if request.policy.cpu_affinity: + # make sure some node has enough CPUs + available = max(self._cpus) + requested = max(request.policy.cpu_affinity) + if requested >= available: + return False, "Cannot satisfy request, not enough CPUs available" + if request.policy.gpu_affinity: + # make sure some node has enough GPUs + available = max(self._gpus) + requested = max(request.policy.gpu_affinity) + if requested >= available: + return False, "Cannot satisfy request, not enough GPUs available" + return True, None + + def _can_honor(self, request: DragonRunRequest) -> t.Tuple[bool, t.Optional[str]]: + """Check if request can be honored with resources available in + the allocation. Currently only checks for total number of nodes, + in the future it will also look at other constraints such as memory, + accelerators, and so on. + + :param request: `DragonRunRequest` to validate + :returns: Tuple indicating if the request can be honored and + an optional error message + """ + honorable, err = self._can_honor_state(request) + if not honorable: + return False, err + + honorable, err = self._can_honor_policy(request) + if not honorable: + return False, err + + honorable, err = self._can_honor_hosts(request) + if not honorable: + return False, err + + honorable, err = self._can_honor_affinities(request) + if not honorable: + return False, err + + return True, None + + def _can_honor_affinities( + self, request: DragonRunRequest + ) -> t.Tuple[bool, t.Optional[str]]: + """Check if the policy can be honored with resources available + in the allocation. + + :param request: the DragonRunRequest to verify + :returns: Tuple indicating if the request can be honored and + an optional error message""" if request.policy: if request.policy.cpu_affinity: # make sure some node has enough CPUs @@ -319,40 +400,96 @@ def _can_honor_policy( return True, None - def _can_honor(self, request: DragonRunRequest) -> t.Tuple[bool, t.Optional[str]]: - """Check if request can be honored with resources available in the allocation. + def _can_honor_hosts( + self, request: DragonRunRequest + ) -> t.Tuple[bool, t.Optional[str]]: + """Check if the current state of the backend process inhibits executing + the request. - Currently only checks for total number of nodes, - in the future it will also look at other constraints - such as memory, accelerators, and so on. - """ + :param request: `DragonRunRequest` to validate + :returns: Tuple indicating if the request can be honored and + an optional error message""" + # fail if requesting more nodes than the total number available if request.nodes > len(self._hosts): + message = f"Cannot satisfy request. {request.nodes} requested nodes" + message += f"exceeds {len(self._hosts)} available." + return False, message + + requested_hosts: t.Set[str] = set(self._hosts) + if request.hostlist: + requested_hosts = {host.strip() for host in request.hostlist.split(",")} + + all_hosts = set(self._hosts) + valid_hosts = all_hosts.intersection(requested_hosts) + invalid_hosts = requested_hosts - valid_hosts + + if invalid_hosts: + logger.warning(f"Some invalid hostnames were requested: {invalid_hosts}") + + # fail if requesting specific hostnames and there aren't enough available + if request.nodes > len(valid_hosts): message = f"Cannot satisfy request. Requested {request.nodes} nodes, " - message += f"but only {len(self._hosts)} nodes are available." + message += f"but only {len(valid_hosts)} named hosts are available." return False, message + + return True, None + + def _can_honor_state( + self, _request: DragonRunRequest + ) -> t.Tuple[bool, t.Optional[str]]: + """Check if the current state of the backend process inhibits executing + the request. + :param _request: the DragonRunRequest to verify + :returns: Tuple indicating if the request can be honored and + an optional error message""" if self._shutdown_requested: message = "Cannot satisfy request, server is shutting down." return False, message - honorable, err = self._can_honor_policy(request) - if not honorable: - return False, err - return True, None def _allocate_step( self, step_id: str, request: DragonRunRequest ) -> t.Optional[t.List[str]]: + """Identify the hosts on which the request will be executed + :param step_id: The identifier of a step that will be executed on the host + :param request: The request to be executed + :returns: A list of selected hostnames""" + # ensure at least one host is selected num_hosts: int = request.nodes + with self._queue_lock: - if num_hosts <= 0 or num_hosts > len(self._free_hosts): + if num_hosts <= 0 or num_hosts > len(self._hosts): + logger.debug( + f"The number of requested hosts ({num_hosts}) is invalid or" + f" cannot be satisfied with {len(self._hosts)} available nodes" + ) return None - to_allocate = [] - for _ in range(num_hosts): - host = self._free_hosts.popleft() + + hosts = [] + if request.hostlist: + # convert the comma-separated argument into a real list + hosts = [host for host in request.hostlist.split(",") if host] + + if hosts: + reference_counts = self._prioritizer.next_n_from(num_hosts, hosts) + else: + filter_on: t.Optional[PrioritizerFilter] = None + if request.policy and request.policy.gpu_affinity: + filter_on = PrioritizerFilter.GPU + reference_counts = self._prioritizer.next_n(num_hosts, filter_on) + + if len(reference_counts) < num_hosts: + # exit if the prioritizer can't identify enough nodes + return None + + to_allocate = [str(ref_counter[1]) for ref_counter in reference_counts] + + # track assigning this step to each node + for host in to_allocate: self._allocated_hosts[host] = step_id - to_allocate.append(host) + return to_allocate @staticmethod @@ -392,6 +529,7 @@ def _create_redirect_workers( return grp_redir def _stop_steps(self) -> None: + """Trigger termination of all currently executing steps""" self._heartbeat() with self._queue_lock: while len(self._stop_requests) > 0: @@ -451,6 +589,7 @@ def create_run_policy( request: DragonRequest, node_name: str ) -> "dragon_policy.Policy": """Create a dragon Policy from the request and node name + :param request: DragonRunRequest containing policy information :param node_name: Name of the node on which the process will run :returns: dragon_policy.Policy object mapped from request properties""" @@ -586,6 +725,8 @@ def _start_steps(self) -> None: logger.error(e) def _refresh_statuses(self) -> None: + """Query underlying management system for step status and update + stored assigned and unassigned task information""" self._heartbeat() with self._queue_lock: terminated = [] @@ -635,10 +776,10 @@ def _refresh_statuses(self) -> None: for host in group_info.hosts: logger.debug(f"Releasing host {host}") try: - self._allocated_hosts.pop(host) + self._allocated_hosts.pop(host) # todo: remove? except KeyError: logger.error(f"Tried to free a non-allocated host: {host}") - self._free_hosts.append(host) + self._prioritizer.decrement(host) group_info.process_group = None group_info.redir_workers = None @@ -662,6 +803,7 @@ def _should_print_status(self) -> bool: return False def _update(self) -> None: + """Trigger all update queries and update local state database""" self._stop_steps() self._start_steps() self._refresh_statuses() @@ -749,8 +891,12 @@ def _(self, request: DragonShutdownRequest) -> DragonShutdownResponse: class DragonBackendView: - def __init__(self, backend: DragonBackend): + def __init__(self, backend: DragonBackend) -> None: + """Initialize the instance + + :param backend: A dragon backend used to produce the view""" self._backend = backend + """A dragon backend used to produce the view""" @property def host_desc(self) -> str: @@ -812,9 +958,7 @@ def step_table(self) -> str: @property def host_table(self) -> str: """Table representation of current state of nodes available - - in the allocation. - """ + in the allocation.""" headers = ["Host", "Status"] hosts = self._backend.hosts free_hosts = self._backend.free_hosts diff --git a/smartsim/_core/launcher/dragon/dragonLauncher.py b/smartsim/_core/launcher/dragon/dragonLauncher.py index 9078fed54..83090bf30 100644 --- a/smartsim/_core/launcher/dragon/dragonLauncher.py +++ b/smartsim/_core/launcher/dragon/dragonLauncher.py @@ -29,7 +29,7 @@ import os import typing as t -from smartsim._core.schemas.dragonRequests import DragonRunPolicy +from smartsim._core.schemas.dragonRequests import HardwarePolicy from ...._core.launcher.stepMapping import StepMap from ....error import LauncherError, SmartSimError @@ -170,8 +170,9 @@ def run(self, step: Step) -> t.Optional[str]: merged_env = self._connector.merge_persisted_env(os.environ.copy()) nodes = int(run_args.get("nodes", None) or 1) tasks_per_node = int(run_args.get("tasks-per-node", None) or 1) + hosts = str(run_args.get("host-list", "")) - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) response = _assert_schema_type( self._connector.send_request( @@ -187,6 +188,7 @@ def run(self, step: Step) -> t.Optional[str]: output_file=out, error_file=err, policy=policy, + hostlist=hosts, ) ), DragonRunResponse, diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py new file mode 100644 index 000000000..3b330b597 --- /dev/null +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -0,0 +1,344 @@ +# BSD 2-Clause License +# +# Copyright (c) 2021-2024, Hewlett Packard Enterprise +# All rights reserved. +# +# Redistribution and use in source and binary forms, with or without +# modification, are permitted provided that the following conditions are met: +# +# 1. Redistributions of source code must retain the above copyright notice, this +# list of conditions and the following disclaimer. +# +# 2. Redistributions in binary form must reproduce the above copyright notice, +# this list of conditions and the following disclaimer in the documentation +# and/or other materials provided with the distribution. +# +# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +# AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +# IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +# DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +# FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +# DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +# SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +# CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +# OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +import enum +import heapq +import threading +import typing as t + +from smartsim.error.errors import SmartSimError +from smartsim.log import get_logger + +logger = get_logger(__name__) + + +# tracking structure for [num_refs, node_name, is_dirty] +_NodeRefCount = t.List[t.Union[int, str]] + + +class PrioritizerFilter(str, enum.Enum): + """A filter used to select a subset of nodes to be queried""" + + CPU = enum.auto() + GPU = enum.auto() + + +class Node(t.Protocol): + """Minimum Node API required to support the NodePrioritizer""" + + @property + def num_cpus(self) -> int: ... + + @property + def num_gpus(self) -> int: ... + + @property + def hostname(self) -> str: ... + + +class NodePrioritizer: + def __init__(self, nodes: t.List[Node], lock: threading.RLock) -> None: + """Initialize the prioritizer + + :param nodes: node attribute information for initializing the priorizer + :param lock: a lock used to ensure threadsafe operations + """ + if not nodes: + raise SmartSimError("Missing nodes to prioritize") + + self._lock = lock + """Lock used to ensure thread safe changes of the reference counters""" + + self._ref_map: t.Dict[str, _NodeRefCount] = {} + """Map node names to a ref counter for direct access""" + + self._cpu_refs: t.List[_NodeRefCount] = [] + """Track reference counts to CPU-only nodes""" + + self._gpu_refs: t.List[_NodeRefCount] = [] + """Track reference counts to GPU nodes""" + + self._initialize_reference_counters(nodes) + + def _initialize_reference_counters(self, nodes: t.List[Node]) -> None: + """Perform initialization of reference counters for nodes in the allocation + + :param nodes: node attribute information for initializing the priorizer""" + for node in nodes: + # initialize all node counts to 0 and mark the entries "is_dirty=False" + tracking_info: _NodeRefCount = [ + 0, + node.hostname, + 0, + ] # use list for mutability + + self._ref_map[node.hostname] = tracking_info + + if node.num_gpus: + self._gpu_refs.append(tracking_info) + else: + self._cpu_refs.append(tracking_info) + + def increment(self, host: str) -> None: + """Directly increment the reference count of a given node and ensure the + ref counter is marked as dirty to trigger a reordering on retrieval + + :param host: a hostname that should have a reference counter selected""" + with self._lock: + tracking_info = self._ref_map[host] + ref_count, *_ = tracking_info + tracking_info[0] = int(ref_count) + 1 + tracking_info[2] = 1 + + def _all_refs(self) -> t.List[_NodeRefCount]: + """Combine the CPU and GPU nodes into a single heap + + :returns: list of all reference counters""" + refs = [*self._cpu_refs, *self._gpu_refs] + heapq.heapify(refs) + return refs + + def get_tracking_info(self, host: str) -> _NodeRefCount: + """Returns the reference counter information for a single node + + :param host: a hostname that should have a reference counter selected + :returns: a reference counter""" + return self._ref_map[host] + + def decrement(self, host: str) -> None: + """Directly increment the reference count of a given node and ensure the + ref counter is marked as dirty to trigger a reordering + + :param host: a hostname that should have a reference counter decremented""" + with self._lock: + tracking_info = self._ref_map[host] + tracking_info[0] = max(int(tracking_info[0]) - 1, 0) + tracking_info[2] = 1 + + def _create_sub_heap(self, hosts: t.List[str]) -> t.List[_NodeRefCount]: + """Create a new heap from the primary heap with user-specified nodes + + :param hosts: a list of hostnames used to filter the available nodes + :returns: a list of assigned reference counters + """ + nodes_tracking_info: t.List[_NodeRefCount] = [] + + # Collect all the tracking info for the requested nodes... + for host in hosts: + tracking_info = self._ref_map[host] + nodes_tracking_info.append(tracking_info) + + # ... and use it to create a new heap from a specified subset of nodes + heapq.heapify(nodes_tracking_info) + + return nodes_tracking_info + + def next_from(self, hosts: t.List[str]) -> t.Optional[_NodeRefCount]: + """Return the next node available given a set of desired hosts + + :param hosts: a list of hostnames used to filter the available nodes + :returns: a list of assigned reference counters + :raises ValueError: if no host names are provided""" + if not hosts or len(hosts) == 0: + raise ValueError("No host names provided") + + sub_heap = self._create_sub_heap(hosts) + return self._get_next_available_node(sub_heap) + + def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.List[_NodeRefCount]: + """Return the next N available nodes given a set of desired hosts + + :param num_items: the desird number of nodes to allocate + :param hosts: a list of hostnames used to filter the available nodes + :returns: a list of reference counts + :raises ValueError: if no host names are provided""" + if not hosts or len(hosts) == 0: + raise ValueError("No host names provided") + + if num_items < 1: + raise ValueError(f"Number of items requested {num_items} is invalid") + + sub_heap = self._create_sub_heap(hosts) + return self._get_next_n_available_nodes(num_items, sub_heap) + + def unassigned( + self, heap: t.Optional[t.List[_NodeRefCount]] = None + ) -> t.List[_NodeRefCount]: + """Select nodes that are currently not assigned a task + + :param heap: (optional) a subset of the node heap to consider + :returns: a list of reference counts for all unassigned nodes""" + if heap is None: + return [node for node in self._ref_map.values() if node[0] == 0] + + return [node for node in heap if node[0] == 0] + + def assigned( + self, heap: t.Optional[t.List[_NodeRefCount]] = None + ) -> t.List[_NodeRefCount]: + """Helper method to identify the nodes that are currently assigned + + :param heap: (optional) a subset of the node heap to consider + :returns: a list of reference counts for all assigned nodes""" + if heap is None: + return [node for node in self._ref_map.values() if node[0] == 1] + + return [node for node in heap if node[0] == 1] + + def _check_satisfiable_n( + self, num_items: int, heap: t.Optional[t.List[_NodeRefCount]] = None + ) -> bool: + """Validates that a request for some number of nodes `n` can be + satisfied by the prioritizer given the set of nodes available + + :param num_items: the desird number of nodes to allocate + :param heap: (optional) a subset of the node heap to consider""" + num_nodes = len(self._ref_map.keys()) + + if num_items < 1: + msg = "Cannot handle request; request requires a positive integer" + logger.warning(msg) + return False + + if num_nodes < num_items: + msg = f"Cannot satisfy request for {num_items} nodes; {num_nodes} in pool" + logger.warning(msg) + return False + + num_open = len(self.unassigned(heap)) + if num_open < num_items: + msg = f"Cannot satisfy request for {num_items} nodes; {num_open} available" + logger.warning(msg) + return False + + return True + + def _get_next_available_node( + self, heap: t.List[_NodeRefCount] + ) -> t.Optional[_NodeRefCount]: + """Finds the next node w/the least amount of running processes and + ensures that any elements that were directly updated are updated in + the priority structure before being made available + + :param heap: (optional) a subset of the node heap to consider + :returns: a reference counter for an available node if an unassigned node + exists, `None` otherwise""" + tracking_info: t.Optional[_NodeRefCount] = None + + with self._lock: + tracking_info = heapq.heappop(heap) + is_dirty = tracking_info[2] + + while is_dirty: + if is_dirty: + # mark dirty items clean and place back into heap to be sorted + tracking_info[2] = 0 + heapq.heappush(heap, tracking_info) + + tracking_info = heapq.heappop(heap) + is_dirty = tracking_info[2] + + original_ref_count = int(tracking_info[0]) + if original_ref_count == 0: + # increment the ref count before putting back onto heap + tracking_info[0] = original_ref_count + 1 + + heapq.heappush(heap, tracking_info) + + # next available must enforce only "open" return nodes + if original_ref_count > 0: + return None + + return tracking_info + + def _get_next_n_available_nodes( + self, + num_items: int, + heap: t.List[_NodeRefCount], + ) -> t.List[_NodeRefCount]: + """Find the next N available nodes w/least amount of references using + the supplied filter to target a specific node capability + + :param n: number of nodes to reserve + :returns: a list of reference counters for a available nodes if enough + unassigned nodes exists, `None` otherwise + :raises ValueError: if the number of requetsed nodes is not a positive integer + """ + next_nodes: t.List[_NodeRefCount] = [] + + if num_items < 1: + raise ValueError(f"Number of items requested {num_items} is invalid") + + if not self._check_satisfiable_n(num_items, heap): + return next_nodes + + while len(next_nodes) < num_items: + next_node = self._get_next_available_node(heap) + if next_node: + next_nodes.append(next_node) + else: + break + + return next_nodes + + def next( + self, filter_on: t.Optional[PrioritizerFilter] = None + ) -> t.Optional[_NodeRefCount]: + """Find the next node available w/least amount of references using + the supplied filter to target a specific node capability + + :param filter_on: the subset of nodes to query for available nodes + :returns: a reference counter for an available node if an unassigned node + exists, `None` otherwise""" + if filter_on == PrioritizerFilter.GPU: + heap = self._gpu_refs + elif filter_on == PrioritizerFilter.CPU: + heap = self._cpu_refs + else: + heap = self._all_refs() + + if node := self._get_next_available_node(heap): + return node + + return None + + def next_n( + self, num_items: int = 1, filter_on: t.Optional[PrioritizerFilter] = None + ) -> t.List[_NodeRefCount]: + """Find the next N available nodes w/least amount of references using + the supplied filter to target a specific node capability + + :param num_items: number of nodes to reserve + :param filter_on: the subset of nodes to query for available nodes + :returns: Collection of reserved nodes""" + heap = self._cpu_refs + if filter_on == PrioritizerFilter.GPU: + heap = self._gpu_refs + elif filter_on == PrioritizerFilter.CPU: + heap = self._cpu_refs + else: + heap = self._all_refs() + + return self._get_next_n_available_nodes(num_items, heap) diff --git a/smartsim/_core/launcher/step/dragonStep.py b/smartsim/_core/launcher/step/dragonStep.py index dd93d7910..331ce2a08 100644 --- a/smartsim/_core/launcher/step/dragonStep.py +++ b/smartsim/_core/launcher/step/dragonStep.py @@ -31,8 +31,8 @@ import typing as t from ...._core.schemas.dragonRequests import ( - DragonRunPolicy, DragonRunRequest, + HardwarePolicy, request_registry, ) from ....error.errors import SSUnsupportedError @@ -167,10 +167,13 @@ def _write_request_file(self) -> str: run_settings = t.cast(DragonRunSettings, step.step_settings) run_args = run_settings.run_args env = run_settings.env_vars - nodes = int(run_args.get("nodes", None) or 1) + nodes = int( + run_args.get("nodes", None) or 1 + ) # todo: is this the same use as my new host-list? tasks_per_node = int(run_args.get("tasks-per-node", None) or 1) + hosts_csv = str(run_args.get("host-list", "")) - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) cmd = step.get_launch_cmd() out, err = step.get_output_files() @@ -187,6 +190,7 @@ def _write_request_file(self) -> str: output_file=out, error_file=err, policy=policy, + hostlist=hosts_csv, ) requests.append(request_registry.to_string(request)) with open(request_file, "w", encoding="utf-8") as script_file: diff --git a/smartsim/_core/schemas/dragonRequests.py b/smartsim/_core/schemas/dragonRequests.py index 487ea915a..c816e6dff 100644 --- a/smartsim/_core/schemas/dragonRequests.py +++ b/smartsim/_core/schemas/dragonRequests.py @@ -40,7 +40,7 @@ class DragonRequest(BaseModel): ... -class DragonRunPolicy(BaseModel): +class HardwarePolicy(BaseModel): """Policy specifying hardware constraints when running a Dragon job""" cpu_affinity: t.List[NonNegativeInt] = Field(default_factory=list) @@ -51,7 +51,7 @@ class DragonRunPolicy(BaseModel): @staticmethod def from_run_args( run_args: t.Dict[str, t.Union[int, str, float, None]] - ) -> "DragonRunPolicy": + ) -> "HardwarePolicy": """Create a DragonRunPolicy with hardware constraints passed from a dictionary of run arguments :param run_args: Dictionary of run arguments @@ -64,12 +64,12 @@ def from_run_args( if cpu_arg_value := run_args.get("cpu-affinity", None): cpu_args = str(cpu_arg_value) - # run args converted to a string must be split back into a list[int] + # list[int] converted to comma-separated str must split into a list[int] gpu_affinity = [int(x.strip()) for x in gpu_args.split(",") if x] cpu_affinity = [int(x.strip()) for x in cpu_args.split(",") if x] try: - return DragonRunPolicy( + return HardwarePolicy( cpu_affinity=cpu_affinity, gpu_affinity=gpu_affinity, ) @@ -95,7 +95,7 @@ class DragonRunRequestView(DragonRequest): @request_registry.register("run") class DragonRunRequest(DragonRunRequestView): current_env: t.Dict[str, t.Optional[str]] = {} - policy: t.Optional[DragonRunPolicy] = None + policy: t.Optional[HardwarePolicy] = None def __str__(self) -> str: return str(DragonRunRequestView.parse_obj(self.dict(exclude={"current_env"}))) diff --git a/smartsim/settings/dragonRunSettings.py b/smartsim/settings/dragonRunSettings.py index 69a91547e..61434b70d 100644 --- a/smartsim/settings/dragonRunSettings.py +++ b/smartsim/settings/dragonRunSettings.py @@ -95,6 +95,14 @@ def set_node_feature(self, feature_list: t.Union[str, t.List[str]]) -> None: self.run_args["node-feature"] = ",".join(feature_list) + @override + def set_hostlist(self, host_list: t.Union[str, t.List[str]]) -> None: + """Specify the hostlist for this job + + :param host_list: hosts to launch on + """ + self.run_args["host-list"] = ",".join(host_list) + def set_cpu_affinity(self, devices: t.List[int]) -> None: """Set the CPU affinity for this job diff --git a/tests/dragon/test_worker_manager.py b/tests/dragon/test_worker_manager.py index c8332c260..d6f5c2ac0 100644 --- a/tests/dragon/test_worker_manager.py +++ b/tests/dragon/test_worker_manager.py @@ -32,6 +32,8 @@ import pytest +from tests.mli.featurestore import FileSystemFeatureStore + torch = pytest.importorskip("torch") dragon = pytest.importorskip("dragon") diff --git a/tests/test_dragon_run_policy.py b/tests/test_dragon_run_policy.py index c94ae375b..83f6fc09a 100644 --- a/tests/test_dragon_run_policy.py +++ b/tests/test_dragon_run_policy.py @@ -159,7 +159,7 @@ def test_create_run_policy_run_request_default_run_policy() -> None: env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(), # <--- passing default values + policy=HardwarePolicy(), # <--- passing default values ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -183,7 +183,7 @@ def test_create_run_policy_run_request_cpu_affinity_no_device() -> None: env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(cpu_affinity=list(affinity)), # <-- no device spec + policy=HardwarePolicy(cpu_affinity=list(affinity)), # <-- no device spec ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -206,7 +206,7 @@ def test_create_run_policy_run_request_cpu_affinity() -> None: env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(cpu_affinity=list(affinity)), + policy=HardwarePolicy(cpu_affinity=list(affinity)), ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -229,7 +229,7 @@ def test_create_run_policy_run_request_gpu_affinity() -> None: env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(device="gpu", gpu_affinity=list(affinity)), + policy=HardwarePolicy(device="gpu", gpu_affinity=list(affinity)), ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -246,7 +246,7 @@ def test_dragon_run_policy_from_run_args() -> None: "cpu-affinity": "3,4,5,6", } - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 5, 6] assert policy.gpu_affinity == [0, 1, 2] @@ -257,7 +257,7 @@ def test_dragon_run_policy_from_run_args_empty() -> None: dictionary of run arguments""" run_args = {} - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [] @@ -270,7 +270,7 @@ def test_dragon_run_policy_from_run_args_cpu_affinity() -> None: "cpu-affinity": "3,4,5,6", } - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 5, 6] assert policy.gpu_affinity == [] @@ -283,7 +283,7 @@ def test_dragon_run_policy_from_run_args_gpu_affinity() -> None: "gpu-affinity": "0, 1, 2", } - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [0, 1, 2] @@ -297,7 +297,7 @@ def test_dragon_run_policy_from_run_args_invalid_gpu_affinity() -> None: } with pytest.raises(SmartSimError) as ex: - DragonRunPolicy.from_run_args(run_args) + HardwarePolicy.from_run_args(run_args) assert "DragonRunPolicy" in ex.value.args[0] @@ -310,7 +310,7 @@ def test_dragon_run_policy_from_run_args_invalid_cpu_affinity() -> None: } with pytest.raises(SmartSimError) as ex: - DragonRunPolicy.from_run_args(run_args) + HardwarePolicy.from_run_args(run_args) assert "DragonRunPolicy" in ex.value.args[0] @@ -322,7 +322,7 @@ def test_dragon_run_policy_from_run_args_ignore_empties_gpu() -> None: "gpu-affinity": "0,,2", } - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [0, 2] @@ -335,7 +335,7 @@ def test_dragon_run_policy_from_run_args_ignore_empties_cpu() -> None: "cpu-affinity": "3,4,,6,", } - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 6] assert policy.gpu_affinity == [] @@ -349,7 +349,7 @@ def test_dragon_run_policy_from_run_args_null_gpu_affinity() -> None: "cpu-affinity": "3,4,5,6", } - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 5, 6] assert policy.gpu_affinity == [] @@ -360,7 +360,7 @@ def test_dragon_run_policy_from_run_args_null_cpu_affinity() -> None: in the cpu-affinity list""" run_args = {"gpu-affinity": "0,1,2", "cpu-affinity": None} - policy = DragonRunPolicy.from_run_args(run_args) + policy = HardwarePolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [0, 1, 2] diff --git a/tests/test_dragon_run_request.py b/tests/test_dragon_run_request.py index 94c17c222..91f8a173f 100644 --- a/tests/test_dragon_run_request.py +++ b/tests/test_dragon_run_request.py @@ -30,8 +30,10 @@ import time from unittest.mock import MagicMock +import pydantic.error_wrappers import pytest -from pydantic import ValidationError + +from smartsim._core.launcher.dragon.pqueue import NodePrioritizer # The tests in this file belong to the group_b group pytestmark = pytest.mark.group_b @@ -162,9 +164,11 @@ def get_mock_backend( from smartsim._core.launcher.dragon.dragonBackend import DragonBackend dragon_backend = DragonBackend(pid=99999) - monkeypatch.setattr( - dragon_backend, "_free_hosts", collections.deque(dragon_backend._hosts) - ) + # monkeypatch.setattr( + # dragon_backend._prioritizer, + # NodePrioritizer(dragon_backend._hosts, dragon_backend._queue_lock), + # collections.deque(dragon_backend._hosts), + # ) return dragon_backend @@ -222,7 +226,6 @@ def set_mock_group_infos( } monkeypatch.setattr(dragon_backend, "_group_infos", group_infos) - monkeypatch.setattr(dragon_backend, "_free_hosts", collections.deque(hosts[1:3])) monkeypatch.setattr(dragon_backend, "_allocated_hosts", {hosts[0]: "abc123-1"}) monkeypatch.setattr(dragon_backend, "_running_steps", ["abc123-1"]) @@ -269,7 +272,7 @@ def test_run_request(monkeypatch: pytest.MonkeyPatch) -> None: assert dragon_backend._running_steps == [step_id] assert len(dragon_backend._queued_steps) == 0 - assert len(dragon_backend._free_hosts) == 1 + assert len(dragon_backend.free_hosts) == 1 assert dragon_backend._allocated_hosts[dragon_backend.hosts[0]] == step_id assert dragon_backend._allocated_hosts[dragon_backend.hosts[1]] == step_id @@ -281,9 +284,11 @@ def test_run_request(monkeypatch: pytest.MonkeyPatch) -> None: assert dragon_backend._running_steps == [step_id] assert len(dragon_backend._queued_steps) == 0 - assert len(dragon_backend._free_hosts) == 1 + assert len(dragon_backend.free_hosts) == 1 assert dragon_backend._allocated_hosts[dragon_backend.hosts[0]] == step_id assert dragon_backend._allocated_hosts[dragon_backend.hosts[1]] == step_id + # assert dragon_backend._assigned_steps[dragon_backend.hosts[0]] == step_id + # assert dragon_backend._assigned_steps[dragon_backend.hosts[1]] == step_id dragon_backend._group_infos[step_id].status = SmartSimStatus.STATUS_CANCELLED @@ -349,7 +354,7 @@ def test_run_request_with_policy(monkeypatch: pytest.MonkeyPatch) -> None: env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(cpu_affinity=[0, 1]), + policy=HardwarePolicy(cpu_affinity=[0, 1]), ) run_resp = dragon_backend.process_request(run_req) @@ -366,7 +371,7 @@ def test_run_request_with_policy(monkeypatch: pytest.MonkeyPatch) -> None: assert dragon_backend._running_steps == [step_id] assert len(dragon_backend._queued_steps) == 0 - assert len(dragon_backend._free_hosts) == 1 + assert len(dragon_backend._prioritizer.unassigned()) == 1 assert dragon_backend._allocated_hosts[dragon_backend.hosts[0]] == step_id assert dragon_backend._allocated_hosts[dragon_backend.hosts[1]] == step_id @@ -378,7 +383,7 @@ def test_run_request_with_policy(monkeypatch: pytest.MonkeyPatch) -> None: assert dragon_backend._running_steps == [step_id] assert len(dragon_backend._queued_steps) == 0 - assert len(dragon_backend._free_hosts) == 1 + assert len(dragon_backend._priorizer.unassigned()) == 1 assert dragon_backend._allocated_hosts[dragon_backend.hosts[0]] == step_id assert dragon_backend._allocated_hosts[dragon_backend.hosts[1]] == step_id @@ -434,7 +439,7 @@ def test_stop_request(monkeypatch: pytest.MonkeyPatch) -> None: ) assert len(dragon_backend._allocated_hosts) == 0 - assert len(dragon_backend._free_hosts) == 3 + assert len(dragon_backend._prioritizer.unassigned()) == 3 @pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @@ -537,9 +542,33 @@ def test_can_honor(monkeypatch: pytest.MonkeyPatch, num_nodes: int) -> None: pmi_enabled=False, ) - assert dragon_backend._can_honor(run_req)[0] == ( - num_nodes <= len(dragon_backend._hosts) - ) + can_honor, error_msg = dragon_backend._can_honor(run_req) + + nodes_in_range = num_nodes <= len(dragon_backend._hosts) + assert can_honor == nodes_in_range + assert error_msg is None if nodes_in_range else error_msg is not None + + +@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") +@pytest.mark.parametrize("num_nodes", [-10, -1, 0]) +def test_can_honor_invalid_num_nodes( + monkeypatch: pytest.MonkeyPatch, num_nodes: int +) -> None: + """Verify that requests for invalid numbers of nodes (negative, zero) are rejected""" + dragon_backend = get_mock_backend(monkeypatch) + + with pytest.raises(pydantic.error_wrappers.ValidationError) as ex: + DragonRunRequest( + exe="sleep", + exe_args=["5"], + path="/a/fake/path", + nodes=num_nodes, + tasks=1, + tasks_per_node=1, + env={}, + current_env={}, + pmi_enabled=False, + ) @pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @@ -559,7 +588,7 @@ def test_can_honor_cpu_affinity( env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(cpu_affinity=affinity), + policy=HardwarePolicy(cpu_affinity=affinity), ) assert dragon_backend._can_honor(run_req)[0] @@ -580,7 +609,7 @@ def test_can_honor_cpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(cpu_affinity=list(range(9))), + policy=HardwarePolicy(cpu_affinity=list(range(9))), ) assert not dragon_backend._can_honor(run_req)[0] @@ -603,7 +632,7 @@ def test_can_honor_gpu_affinity( env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(gpu_affinity=affinity), + policy=HardwarePolicy(gpu_affinity=affinity), ) assert dragon_backend._can_honor(run_req)[0] @@ -624,7 +653,7 @@ def test_can_honor_gpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(gpu_affinity=list(range(3))), + policy=HardwarePolicy(gpu_affinity=list(range(3))), ) assert not dragon_backend._can_honor(run_req)[0] @@ -648,7 +677,7 @@ def test_can_honor_gpu_device_not_available(monkeypatch: pytest.MonkeyPatch) -> current_env={}, pmi_enabled=False, # specify GPU device w/no affinity - policy=DragonRunPolicy(gpu_affinity=[0]), + policy=HardwarePolicy(gpu_affinity=[0]), ) assert not dragon_backend._can_honor(run_req)[0] @@ -669,23 +698,92 @@ def test_view(monkeypatch: pytest.MonkeyPatch) -> None: set_mock_group_infos(monkeypatch, dragon_backend) hosts = dragon_backend.hosts - expected_message = textwrap.dedent(f"""\ + expected_message = textwrap.dedent( + f"""\ Dragon server backend update | Host | Status | - |--------|----------| + |---------|----------| | {hosts[0]} | Busy | | {hosts[1]} | Free | | {hosts[2]} | Free | | Step | Status | Hosts | Return codes | Num procs | - |----------|--------------|-------------|----------------|-------------| + |----------|--------------|-----------------|----------------|-------------| | abc123-1 | Running | {hosts[0]} | | 1 | | del999-2 | Cancelled | {hosts[1]} | -9 | 1 | | c101vz-3 | Completed | {hosts[1]},{hosts[2]} | 0 | 2 | | 0ghjk1-4 | Failed | {hosts[2]} | -1 | 1 | - | ljace0-5 | NeverStarted | | | 0 |""") + | ljace0-5 | NeverStarted | | | 0 |""" + ) # get rid of white space to make the comparison easier actual_msg = dragon_backend.status_message.replace(" ", "") expected_message = expected_message.replace(" ", "") assert actual_msg == expected_message + + +def test_can_honor_hosts_unavailable_hosts(monkeypatch: pytest.MonkeyPatch) -> None: + """Verify that requesting nodes with invalid names causes number of available + nodes check to fail due to valid # of named nodes being under num_nodes""" + dragon_backend = get_mock_backend(monkeypatch) + + # let's supply 2 invalid and 1 valid hostname + actual_hosts = list(dragon_backend._hosts) + actual_hosts[0] = f"x{actual_hosts[0]}" + actual_hosts[1] = f"x{actual_hosts[1]}" + + host_list = ",".join(actual_hosts) + + run_req = DragonRunRequest( + exe="sleep", + exe_args=["5"], + path="/a/fake/path", + nodes=2, # <----- requesting 2 of 3 available nodes + hostlist=host_list, # <--- only one valid name available + tasks=1, + tasks_per_node=1, + env={}, + current_env={}, + pmi_enabled=False, + policy=HardwarePolicy(), + ) + + can_honor, error_msg = dragon_backend._can_honor(run_req) + + # confirm the failure is indicated + assert not can_honor + # confirm failure message indicates number of nodes requested as cause + assert "named hosts" in error_msg + + +def test_can_honor_hosts_unavailable_hosts_ok(monkeypatch: pytest.MonkeyPatch) -> None: + """Verify that requesting nodes with invalid names causes number of available + nodes check to be reduced but still passes if enough valid named nodes are passed""" + dragon_backend = get_mock_backend(monkeypatch) + + # let's supply 2 valid and 1 invalid hostname + actual_hosts = list(dragon_backend._hosts) + actual_hosts[0] = f"x{actual_hosts[0]}" + + host_list = ",".join(actual_hosts) + + run_req = DragonRunRequest( + exe="sleep", + exe_args=["5"], + path="/a/fake/path", + nodes=2, # <----- requesting 2 of 3 available nodes + hostlist=host_list, # <--- two valid names are available + tasks=1, + tasks_per_node=1, + env={}, + current_env={}, + pmi_enabled=False, + policy=HardwarePolicy(), + ) + + can_honor, error_msg = dragon_backend._can_honor(run_req) + + # confirm the failure is indicated + assert can_honor, error_msg + # confirm failure message indicates number of nodes requested as cause + assert error_msg is None, error_msg diff --git a/tests/test_dragon_run_request_nowlm.py b/tests/test_dragon_run_request_nowlm.py index afd25aa9d..2cacc3a9d 100644 --- a/tests/test_dragon_run_request_nowlm.py +++ b/tests/test_dragon_run_request_nowlm.py @@ -65,7 +65,7 @@ def test_run_request_with_empty_policy(monkeypatch: pytest.MonkeyPatch) -> None: env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy(), + policy=HardwarePolicy(), ) assert run_req.policy is not None assert not run_req.policy.cpu_affinity @@ -96,9 +96,7 @@ def test_run_request_with_negative_affinity( env={}, current_env={}, pmi_enabled=False, - policy=DragonRunPolicy( - cpu_affinity=cpu_affinity, gpu_affinity=gpu_affinity - ), + policy=HardwarePolicy(cpu_affinity=cpu_affinity, gpu_affinity=gpu_affinity), ) assert f"{device}_affinity" in str(ex.value.args[0]) diff --git a/tests/test_node_prioritizer.py b/tests/test_node_prioritizer.py new file mode 100644 index 000000000..12668ed21 --- /dev/null +++ b/tests/test_node_prioritizer.py @@ -0,0 +1,550 @@ +# BSD 2-Clause License +# +# Copyright (c) 2021-2024, Hewlett Packard Enterprise +# All rights reserved. +# +# Redistribution and use in source and binary forms, with or without +# modification, are permitted provided that the following conditions are met: +# +# 1. Redistributions of source code must retain the above copyright notice, this +# list of conditions and the following disclaimer. +# +# 2. Redistributions in binary form must reproduce the above copyright notice, +# this list of conditions and the following disclaimer in the documentation +# and/or other materials provided with the distribution. +# +# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +# AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +# IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +# DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +# FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +# DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +# SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +# CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +# OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +import random +import threading +import typing as t + +import pytest + +from smartsim._core.launcher.dragon.pqueue import NodePrioritizer, PrioritizerFilter +from smartsim.error.errors import SmartSimError +from smartsim.log import get_logger + +# The tests in this file belong to the group_b group +pytestmark = pytest.mark.group_b + + +logger = get_logger(__name__) + + +class MockNode: + def __init__(self, num_cpus: int, num_gpus: int, hostname: str) -> None: + self.num_cpus = num_cpus + self.num_gpus = num_gpus + self.hostname = hostname + + +def mock_node_hosts( + num_cpu_nodes: int, num_gpu_nodes: int +) -> t.Tuple[t.List[MockNode], t.List[MockNode]]: + cpu_hosts = [f"cpu-node-{i}" for i in range(num_cpu_nodes)] + gpu_hosts = [f"gpu-node-{i}" for i in range(num_gpu_nodes)] + + return cpu_hosts, gpu_hosts + + +def mock_node_builder(num_cpu_nodes: int, num_gpu_nodes: int) -> t.List[MockNode]: + nodes = [] + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + + nodes.extend(MockNode(4, 0, hostname) for hostname in cpu_hosts) + nodes.extend(MockNode(4, 4, hostname) for hostname in gpu_hosts) + + return nodes + + +def test_node_prioritizer_init_null() -> None: + """Verify that the priorizer reports failures to send a valid node set + if a null value is passed""" + lock = threading.RLock() + with pytest.raises(SmartSimError) as ex: + NodePrioritizer(None, lock) + + assert "Missing" in ex.value.args[0] + + +def test_node_prioritizer_init_empty() -> None: + """Verify that the priorizer reports failures to send a valid node set + if an empty list is passed""" + lock = threading.RLock() + with pytest.raises(SmartSimError) as ex: + NodePrioritizer([], lock) + + assert "Missing" in ex.value.args[0] + + +@pytest.mark.parametrize( + "num_cpu_nodes,num_gpu_nodes", [(1, 1), (2, 1), (1, 2), (8, 4), (1000, 200)] +) +def test_node_prioritizer_init_ok(num_cpu_nodes: int, num_gpu_nodes: int) -> None: + """Verify that initialization with a valid node list results in the + appropriate cpu & gpu ref counts, and complete ref map""" + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + # perform prioritizer initialization + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # get a copy of all the expected host names + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + all_hosts = cpu_hosts + gpu_hosts + assert len(all_hosts) == num_cpu_nodes + num_gpu_nodes + + # verify tracking data is initialized correctly for all nodes + for hostname in all_hosts: + # show that the ref map is tracking the node + assert hostname in p._ref_map + + tracking_info = p._ref_map[hostname] + + # show that the node is created w/zero ref counts + assert tracking_info[0] == 0 + + # show that the node is created and marked as not dirty (unchanged) + assert tracking_info[2] == False + + # iterate through known cpu node keys and verify prioritizer initialization + for hostname in cpu_hosts: + # show that the device ref counters are appropriately assigned + cpu_ref = next((n for n in p._cpu_refs if n[1] == hostname), None) + assert cpu_ref, "CPU-only node not found in cpu ref set" + + gpu_ref = next((n for n in p._gpu_refs if n[1] == hostname), None) + assert not gpu_ref, "CPU-only node should not be found in gpu ref set" + + # iterate through known GPU node keys and verify prioritizer initialization + for hostname in gpu_hosts: + # show that the device ref counters are appropriately assigned + gpu_ref = next((n for n in p._gpu_refs if n[1] == hostname), None) + assert gpu_ref, "GPU-only node not found in gpu ref set" + + cpu_ref = next((n for n in p._cpu_refs if n[1] == hostname), None) + assert not cpu_ref, "GPU-only node should not be found in cpu ref set" + + # verify we have all hosts in the ref map + assert set(p._ref_map.keys()) == set(all_hosts) + + # verify we have no extra hosts in ref map + assert len(p._ref_map.keys()) == len(set(all_hosts)) + + +def test_node_prioritizer_direct_increment() -> None: + """Verify that performing the increment operation causes the expected + side effect on the intended records""" + + num_cpu_nodes, num_gpu_nodes = 32, 8 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + exclude_index = 2 + exclude_host0 = cpu_hosts[exclude_index] + exclude_host1 = gpu_hosts[exclude_index] + exclusions = [exclude_host0, exclude_host1] + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # let's increment each element in a predictable way and verify + for node in nodes: + if node.hostname in exclusions: + # expect 1 cpu and 1 gpu node at zero and not incremented + continue + + if node.num_gpus == 0: + num_increments = random.randint(0, num_cpu_nodes - 1) + else: + num_increments = random.randint(0, num_gpu_nodes - 1) + + # increment this node some random number of times + for _ in range(num_increments): + p.increment(node.hostname) + + # ... and verify the correct incrementing is applied + tracking_info = p.get_tracking_info(node.hostname) + assert tracking_info[0] == num_increments + + # verify the excluded cpu node was never changed + tracking_info0 = p.get_tracking_info(exclude_host0) + assert tracking_info0[0] == 0 + + # verify the excluded gpu node was never changed + tracking_info1 = p.get_tracking_info(exclude_host1) + assert tracking_info1[0] == 0 + + +def test_node_prioritizer_indirect_increment() -> None: + """Verify that performing the increment operation indirectly affects + each available node until we run out of nodes to return""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # perform indirect + for i in range(len(nodes)): + tracking_info = p.get_tracking_info(nodes[i].hostname) + assert tracking_info[0] == 0 # <--- ref count starts at zero + + # apply operation + tracking_info = p.next(PrioritizerFilter.CPU) + + # verify side-effects + assert tracking_info[0] > 0 # <--- ref count should now be > 0 + assert ( + tracking_info[2] == False + ) # <--- we expect it to give back only "clean" nodes + + # every node has been incremented now. prioritizer shouldn't have anything to give + tracking_info = p.next(PrioritizerFilter.CPU) + assert tracking_info is None # <--- get_next shouldn't have any nodes to give + + +def test_node_prioritizer_indirect_decrement_availability() -> None: + """Verify that a node who is decremented (dirty) is made assignable + on a subsequent request""" + + num_cpu_nodes, num_gpu_nodes = 1, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # increment our only node... + p.increment(cpu_hosts[0]) + + tracking_info = p.next() + assert tracking_info is None, "No nodes should be assignable" + + # perform a decrement... + p.decrement(cpu_hosts[0]) + + # ... and confirm that the node is available again + tracking_info = p.next() + assert tracking_info is not None, "A node should be assignable" + + +def test_node_prioritizer_multi_increment() -> None: + """Verify that retrieving multiple nodes via `next_n` API correctly + increments reference counts and returns appropriate results""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # Mark some nodes as dirty to verify retrieval + p.increment(cpu_hosts[0]) + assert p.get_tracking_info(cpu_hosts[0])[0] > 0 + + p.increment(cpu_hosts[2]) + assert p.get_tracking_info(cpu_hosts[2])[0] > 0 + + p.increment(cpu_hosts[4]) + assert p.get_tracking_info(cpu_hosts[4])[0] > 0 + + # use next_n w/the minimum allowed value + all_tracking_info = p.next_n(1, PrioritizerFilter.CPU) # <---- next_n(1) + + # confirm the number requested is honored and the expected node is returned + assert len(all_tracking_info) == 1 + assert all_tracking_info[0][1] == cpu_hosts[1] + + # use next_n w/value that exceeds available number of open nodes + all_tracking_info = p.next_n( + 5, PrioritizerFilter.CPU + ) # <---- 3 direct increments in setup, 1 out of next_n(1), 4 left + + # confirm that no nodes are returned, even though 4 out of 5 requested are available + assert len(all_tracking_info) == 0 + + +def test_node_prioritizer_multi_increment_validate_n() -> None: + """Verify that retrieving multiple nodes via `next_n` API correctly + reports failures when the request size is above pool size""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # # request 0 nodes, ensure we don't break + # all_tracking_info = p.next_n(0, PrioritizerFilter.CPU) + # assert len(all_tracking_info) == 0 + + # # request negative node count, confirm we don't break + # all_tracking_info = p.next_n(-1, PrioritizerFilter.CPU) + # assert len(all_tracking_info) == 0 + + # we have 8 total cpu nodes available... request too many nodes + all_tracking_info = p.next_n(9, PrioritizerFilter.CPU) + assert len(all_tracking_info) == 0 + + all_tracking_info = p.next_n(num_cpu_nodes * 1000, PrioritizerFilter.CPU) + assert len(all_tracking_info) == 0 + + +def test_node_prioritizer_indirect_direct_interleaved_increments() -> None: + """Verify that interleaving indirect and direct increments results in + expected ref counts""" + + num_cpu_nodes, num_gpu_nodes = 8, 4 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # perform some set of non-popped increments + p.increment(gpu_hosts[1]) + p.increment(gpu_hosts[3]) + p.increment(gpu_hosts[3]) + + # increment 0th item 1x + p.increment(cpu_hosts[0]) + + # increment 3th item 2x + p.increment(cpu_hosts[3]) + p.increment(cpu_hosts[3]) + + # increment last item 3x + p.increment(cpu_hosts[7]) + p.increment(cpu_hosts[7]) + p.increment(cpu_hosts[7]) + + tracking_info = p.get_tracking_info(gpu_hosts[1]) + assert tracking_info[0] == 1 + + tracking_info = p.get_tracking_info(gpu_hosts[3]) + assert tracking_info[0] == 2 + + # we should skip the 0-th item in the heap due to direct increment + tracking_info = p.next(PrioritizerFilter.CPU) + assert tracking_info[0] == 1 + assert tracking_info[1] == cpu_hosts[1] + + # this should pull the next item right out + tracking_info = p.next(PrioritizerFilter.CPU) + assert tracking_info[0] == 1 + assert tracking_info[1] == cpu_hosts[2] + + # we should step over the 3-th node on this iteration + tracking_info = p.next(PrioritizerFilter.CPU) + assert tracking_info[0] == 1 + assert tracking_info[1] == cpu_hosts[4] + + # ensure we pull from gpu nodes and the 0th item is returned + tracking_info = p.next(PrioritizerFilter.GPU) + assert tracking_info[0] == 1 + assert tracking_info[1] == gpu_hosts[0] + + # and ensure that heap also steps over a direct increment + tracking_info = p.next(PrioritizerFilter.GPU) + assert tracking_info[0] == 1 + assert tracking_info[1] == gpu_hosts[2] + + # and another GPU request should return nothing + tracking_info = p.next(PrioritizerFilter.GPU) + assert tracking_info is None + + +def test_node_prioritizer_decrement_floor() -> None: + """Verify that repeatedly decrementing ref counts does not + allow negative ref counts""" + + num_cpu_nodes, num_gpu_nodes = 8, 4 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # try a ton of decrements on all the items in the prioritizer + for _ in range(len(nodes) * 100): + index = random.randint(0, num_cpu_nodes - 1) + p.decrement(cpu_hosts[index]) + + index = random.randint(0, num_gpu_nodes - 1) + p.decrement(gpu_hosts[index]) + + for node in nodes: + tracking_info = p.get_tracking_info(node.hostname) + assert tracking_info[0] == 0 + + +@pytest.mark.parametrize("num_requested", [1, 2, 3]) +def test_node_prioritizer_multi_increment_subheap(num_requested: int) -> None: + """Verify that retrieving multiple nodes via `next_n_from` API correctly + increments reference counts and returns appropriate results + when requesting an in-bounds number of nodes""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # Mark some nodes as dirty to verify retrieval + p.increment(cpu_hosts[0]) + p.increment(cpu_hosts[2]) + p.increment(cpu_hosts[4]) + + hostnames = [cpu_hosts[0], cpu_hosts[1], cpu_hosts[2], cpu_hosts[3], cpu_hosts[5]] + + # request n == {num_requested} nodes from set of 3 available + all_tracking_info = p.next_n_from( + num_requested, + hostnames, + ) # <---- w/0,2,4 assigned, only 1,3,5 from hostnames can work + + # all parameterizations should result in a matching output size + assert len(all_tracking_info) == num_requested + + +def test_node_prioritizer_multi_increment_subheap_assigned() -> None: + """Verify that retrieving multiple nodes via `next_n_from` API does + not return anything when the number requested cannot be satisfied + by the given subheap due to prior assignment""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # Mark some nodes as dirty to verify retrieval + p.increment(cpu_hosts[0]) + p.increment(cpu_hosts[2]) + + hostnames = [cpu_hosts[0], cpu_hosts[2]] + + # request n == {num_requested} nodes from set of 3 available + num_requested = 1 + all_tracking_info = p.next_n_from(num_requested, hostnames) + + # w/0,2 assigned, nothing can be returned + assert len(all_tracking_info) == 0 + + +def test_node_prioritizer_empty_subheap_next_from() -> None: + """Verify that retrieving multiple nodes via `next_n_from` API does + not allow an empty host list""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # Mark some nodes as dirty to verify retrieval + p.increment(cpu_hosts[0]) + p.increment(cpu_hosts[2]) + + # hostnames = [cpu_hosts[0], cpu_hosts[2]] + hostnames = [] + + # request n == {num_requested} nodes from set of 3 available + num_requested = 1 + with pytest.raises(ValueError) as ex: + p.next_from(hostnames) + + assert "No host names provided" == ex.value.args[0] + + +def test_node_prioritizer_empty_subheap_next_n_from() -> None: + """Verify that retrieving multiple nodes via `next_n_from` API does + not allow an empty host list""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # Mark some nodes as dirty to verify retrieval + p.increment(cpu_hosts[0]) + p.increment(cpu_hosts[2]) + + # hostnames = [cpu_hosts[0], cpu_hosts[2]] + hostnames = [] + + # request n == {num_requested} nodes from set of 3 available + num_requested = 1 + with pytest.raises(ValueError) as ex: + p.next_n_from(num_requested, hostnames) + + assert "No host names provided" == ex.value.args[0] + + +@pytest.mark.parametrize("num_requested", [-100, -1, 0]) +def test_node_prioritizer_empty_subheap_next_n(num_requested: int) -> None: + """Verify that retrieving a node via `next_n` API does + not allow a request with num_items < 1""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # Mark some nodes as dirty to verify retrieval + p.increment(cpu_hosts[0]) + p.increment(cpu_hosts[2]) + + # hostnames = [cpu_hosts[0], cpu_hosts[2]] + + # request n == {num_requested} nodes from set of 3 available + with pytest.raises(ValueError) as ex: + p.next_n(num_requested) + + assert "Number of items requested" in ex.value.args[0] + + +@pytest.mark.parametrize("num_requested", [-100, -1, 0]) +def test_node_prioritizer_empty_subheap_next_n_from(num_requested: int) -> None: + """Verify that retrieving multiple nodes via `next_n_from` API does + not allow a request with num_items < 1""" + + num_cpu_nodes, num_gpu_nodes = 8, 0 + cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) + nodes = mock_node_builder(num_cpu_nodes, num_gpu_nodes) + + lock = threading.RLock() + p = NodePrioritizer(nodes, lock) + + # Mark some nodes as dirty to verify retrieval + p.increment(cpu_hosts[0]) + p.increment(cpu_hosts[2]) + + hostnames = [cpu_hosts[0], cpu_hosts[2]] + + # request n == {num_requested} nodes from set of 3 available + with pytest.raises(ValueError) as ex: + p.next_n_from(num_requested, hostnames) + + assert "Number of items requested" in ex.value.args[0] From b4224a63c3c63e5412896f821f245a56515a3f84 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Wed, 7 Aug 2024 20:21:26 -0500 Subject: [PATCH 02/24] remove commented code --- tests/test_node_prioritizer.py | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/test_node_prioritizer.py b/tests/test_node_prioritizer.py index 12668ed21..420308979 100644 --- a/tests/test_node_prioritizer.py +++ b/tests/test_node_prioritizer.py @@ -288,14 +288,6 @@ def test_node_prioritizer_multi_increment_validate_n() -> None: lock = threading.RLock() p = NodePrioritizer(nodes, lock) - # # request 0 nodes, ensure we don't break - # all_tracking_info = p.next_n(0, PrioritizerFilter.CPU) - # assert len(all_tracking_info) == 0 - - # # request negative node count, confirm we don't break - # all_tracking_info = p.next_n(-1, PrioritizerFilter.CPU) - # assert len(all_tracking_info) == 0 - # we have 8 total cpu nodes available... request too many nodes all_tracking_info = p.next_n(9, PrioritizerFilter.CPU) assert len(all_tracking_info) == 0 @@ -463,7 +455,6 @@ def test_node_prioritizer_empty_subheap_next_from() -> None: p.increment(cpu_hosts[0]) p.increment(cpu_hosts[2]) - # hostnames = [cpu_hosts[0], cpu_hosts[2]] hostnames = [] # request n == {num_requested} nodes from set of 3 available @@ -489,7 +480,6 @@ def test_node_prioritizer_empty_subheap_next_n_from() -> None: p.increment(cpu_hosts[0]) p.increment(cpu_hosts[2]) - # hostnames = [cpu_hosts[0], cpu_hosts[2]] hostnames = [] # request n == {num_requested} nodes from set of 3 available @@ -516,8 +506,6 @@ def test_node_prioritizer_empty_subheap_next_n(num_requested: int) -> None: p.increment(cpu_hosts[0]) p.increment(cpu_hosts[2]) - # hostnames = [cpu_hosts[0], cpu_hosts[2]] - # request n == {num_requested} nodes from set of 3 available with pytest.raises(ValueError) as ex: p.next_n(num_requested) From b7cb428020f99e627598c720cdf2b86ce01de56f Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Thu, 8 Aug 2024 10:39:33 -0500 Subject: [PATCH 03/24] undo rename --- .../_core/launcher/dragon/dragonLauncher.py | 4 +-- smartsim/_core/launcher/step/dragonStep.py | 4 +-- smartsim/_core/schemas/dragonRequests.py | 8 +++--- tests/test_dragon_run_policy.py | 28 +++++++++---------- tests/test_dragon_run_request.py | 16 +++++------ tests/test_dragon_run_request_nowlm.py | 6 ++-- 6 files changed, 34 insertions(+), 32 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonLauncher.py b/smartsim/_core/launcher/dragon/dragonLauncher.py index 83090bf30..e8391410b 100644 --- a/smartsim/_core/launcher/dragon/dragonLauncher.py +++ b/smartsim/_core/launcher/dragon/dragonLauncher.py @@ -29,7 +29,7 @@ import os import typing as t -from smartsim._core.schemas.dragonRequests import HardwarePolicy +from smartsim._core.schemas.dragonRequests import DragonRunPolicy from ...._core.launcher.stepMapping import StepMap from ....error import LauncherError, SmartSimError @@ -172,7 +172,7 @@ def run(self, step: Step) -> t.Optional[str]: tasks_per_node = int(run_args.get("tasks-per-node", None) or 1) hosts = str(run_args.get("host-list", "")) - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) response = _assert_schema_type( self._connector.send_request( diff --git a/smartsim/_core/launcher/step/dragonStep.py b/smartsim/_core/launcher/step/dragonStep.py index 331ce2a08..bca693ce7 100644 --- a/smartsim/_core/launcher/step/dragonStep.py +++ b/smartsim/_core/launcher/step/dragonStep.py @@ -32,7 +32,7 @@ from ...._core.schemas.dragonRequests import ( DragonRunRequest, - HardwarePolicy, + DragonRunPolicy, request_registry, ) from ....error.errors import SSUnsupportedError @@ -173,7 +173,7 @@ def _write_request_file(self) -> str: tasks_per_node = int(run_args.get("tasks-per-node", None) or 1) hosts_csv = str(run_args.get("host-list", "")) - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) cmd = step.get_launch_cmd() out, err = step.get_output_files() diff --git a/smartsim/_core/schemas/dragonRequests.py b/smartsim/_core/schemas/dragonRequests.py index c816e6dff..636375faf 100644 --- a/smartsim/_core/schemas/dragonRequests.py +++ b/smartsim/_core/schemas/dragonRequests.py @@ -40,7 +40,7 @@ class DragonRequest(BaseModel): ... -class HardwarePolicy(BaseModel): +class DragonRunPolicy(BaseModel): """Policy specifying hardware constraints when running a Dragon job""" cpu_affinity: t.List[NonNegativeInt] = Field(default_factory=list) @@ -51,7 +51,7 @@ class HardwarePolicy(BaseModel): @staticmethod def from_run_args( run_args: t.Dict[str, t.Union[int, str, float, None]] - ) -> "HardwarePolicy": + ) -> "DragonRunPolicy": """Create a DragonRunPolicy with hardware constraints passed from a dictionary of run arguments :param run_args: Dictionary of run arguments @@ -69,7 +69,7 @@ def from_run_args( cpu_affinity = [int(x.strip()) for x in cpu_args.split(",") if x] try: - return HardwarePolicy( + return DragonRunPolicy( cpu_affinity=cpu_affinity, gpu_affinity=gpu_affinity, ) @@ -95,7 +95,7 @@ class DragonRunRequestView(DragonRequest): @request_registry.register("run") class DragonRunRequest(DragonRunRequestView): current_env: t.Dict[str, t.Optional[str]] = {} - policy: t.Optional[HardwarePolicy] = None + policy: t.Optional[DragonRunPolicy] = None def __str__(self) -> str: return str(DragonRunRequestView.parse_obj(self.dict(exclude={"current_env"}))) diff --git a/tests/test_dragon_run_policy.py b/tests/test_dragon_run_policy.py index 83f6fc09a..c94ae375b 100644 --- a/tests/test_dragon_run_policy.py +++ b/tests/test_dragon_run_policy.py @@ -159,7 +159,7 @@ def test_create_run_policy_run_request_default_run_policy() -> None: env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(), # <--- passing default values + policy=DragonRunPolicy(), # <--- passing default values ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -183,7 +183,7 @@ def test_create_run_policy_run_request_cpu_affinity_no_device() -> None: env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(cpu_affinity=list(affinity)), # <-- no device spec + policy=DragonRunPolicy(cpu_affinity=list(affinity)), # <-- no device spec ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -206,7 +206,7 @@ def test_create_run_policy_run_request_cpu_affinity() -> None: env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(cpu_affinity=list(affinity)), + policy=DragonRunPolicy(cpu_affinity=list(affinity)), ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -229,7 +229,7 @@ def test_create_run_policy_run_request_gpu_affinity() -> None: env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(device="gpu", gpu_affinity=list(affinity)), + policy=DragonRunPolicy(device="gpu", gpu_affinity=list(affinity)), ) policy = DragonBackend.create_run_policy(run_req, "localhost") @@ -246,7 +246,7 @@ def test_dragon_run_policy_from_run_args() -> None: "cpu-affinity": "3,4,5,6", } - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 5, 6] assert policy.gpu_affinity == [0, 1, 2] @@ -257,7 +257,7 @@ def test_dragon_run_policy_from_run_args_empty() -> None: dictionary of run arguments""" run_args = {} - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [] @@ -270,7 +270,7 @@ def test_dragon_run_policy_from_run_args_cpu_affinity() -> None: "cpu-affinity": "3,4,5,6", } - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 5, 6] assert policy.gpu_affinity == [] @@ -283,7 +283,7 @@ def test_dragon_run_policy_from_run_args_gpu_affinity() -> None: "gpu-affinity": "0, 1, 2", } - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [0, 1, 2] @@ -297,7 +297,7 @@ def test_dragon_run_policy_from_run_args_invalid_gpu_affinity() -> None: } with pytest.raises(SmartSimError) as ex: - HardwarePolicy.from_run_args(run_args) + DragonRunPolicy.from_run_args(run_args) assert "DragonRunPolicy" in ex.value.args[0] @@ -310,7 +310,7 @@ def test_dragon_run_policy_from_run_args_invalid_cpu_affinity() -> None: } with pytest.raises(SmartSimError) as ex: - HardwarePolicy.from_run_args(run_args) + DragonRunPolicy.from_run_args(run_args) assert "DragonRunPolicy" in ex.value.args[0] @@ -322,7 +322,7 @@ def test_dragon_run_policy_from_run_args_ignore_empties_gpu() -> None: "gpu-affinity": "0,,2", } - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [0, 2] @@ -335,7 +335,7 @@ def test_dragon_run_policy_from_run_args_ignore_empties_cpu() -> None: "cpu-affinity": "3,4,,6,", } - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 6] assert policy.gpu_affinity == [] @@ -349,7 +349,7 @@ def test_dragon_run_policy_from_run_args_null_gpu_affinity() -> None: "cpu-affinity": "3,4,5,6", } - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [3, 4, 5, 6] assert policy.gpu_affinity == [] @@ -360,7 +360,7 @@ def test_dragon_run_policy_from_run_args_null_cpu_affinity() -> None: in the cpu-affinity list""" run_args = {"gpu-affinity": "0,1,2", "cpu-affinity": None} - policy = HardwarePolicy.from_run_args(run_args) + policy = DragonRunPolicy.from_run_args(run_args) assert policy.cpu_affinity == [] assert policy.gpu_affinity == [0, 1, 2] diff --git a/tests/test_dragon_run_request.py b/tests/test_dragon_run_request.py index 91f8a173f..4545517ec 100644 --- a/tests/test_dragon_run_request.py +++ b/tests/test_dragon_run_request.py @@ -354,7 +354,7 @@ def test_run_request_with_policy(monkeypatch: pytest.MonkeyPatch) -> None: env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(cpu_affinity=[0, 1]), + policy=DragonRunPolicy(cpu_affinity=[0, 1]), ) run_resp = dragon_backend.process_request(run_req) @@ -588,7 +588,7 @@ def test_can_honor_cpu_affinity( env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(cpu_affinity=affinity), + policy=DragonRunPolicy(cpu_affinity=affinity), ) assert dragon_backend._can_honor(run_req)[0] @@ -609,7 +609,7 @@ def test_can_honor_cpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(cpu_affinity=list(range(9))), + policy=DragonRunPolicy(cpu_affinity=list(range(9))), ) assert not dragon_backend._can_honor(run_req)[0] @@ -632,7 +632,7 @@ def test_can_honor_gpu_affinity( env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(gpu_affinity=affinity), + policy=DragonRunPolicy(gpu_affinity=affinity), ) assert dragon_backend._can_honor(run_req)[0] @@ -653,7 +653,7 @@ def test_can_honor_gpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(gpu_affinity=list(range(3))), + policy=DragonRunPolicy(gpu_affinity=list(range(3))), ) assert not dragon_backend._can_honor(run_req)[0] @@ -677,7 +677,7 @@ def test_can_honor_gpu_device_not_available(monkeypatch: pytest.MonkeyPatch) -> current_env={}, pmi_enabled=False, # specify GPU device w/no affinity - policy=HardwarePolicy(gpu_affinity=[0]), + policy=DragonRunPolicy(gpu_affinity=[0]), ) assert not dragon_backend._can_honor(run_req)[0] @@ -745,7 +745,7 @@ def test_can_honor_hosts_unavailable_hosts(monkeypatch: pytest.MonkeyPatch) -> N env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(), + policy=DragonRunPolicy(), ) can_honor, error_msg = dragon_backend._can_honor(run_req) @@ -778,7 +778,7 @@ def test_can_honor_hosts_unavailable_hosts_ok(monkeypatch: pytest.MonkeyPatch) - env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(), + policy=DragonRunPolicy(), ) can_honor, error_msg = dragon_backend._can_honor(run_req) diff --git a/tests/test_dragon_run_request_nowlm.py b/tests/test_dragon_run_request_nowlm.py index 2cacc3a9d..afd25aa9d 100644 --- a/tests/test_dragon_run_request_nowlm.py +++ b/tests/test_dragon_run_request_nowlm.py @@ -65,7 +65,7 @@ def test_run_request_with_empty_policy(monkeypatch: pytest.MonkeyPatch) -> None: env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(), + policy=DragonRunPolicy(), ) assert run_req.policy is not None assert not run_req.policy.cpu_affinity @@ -96,7 +96,9 @@ def test_run_request_with_negative_affinity( env={}, current_env={}, pmi_enabled=False, - policy=HardwarePolicy(cpu_affinity=cpu_affinity, gpu_affinity=gpu_affinity), + policy=DragonRunPolicy( + cpu_affinity=cpu_affinity, gpu_affinity=gpu_affinity + ), ) assert f"{device}_affinity" in str(ex.value.args[0]) From 1fb869d8c223f132eb922a24a6544ff7a2dfa5cb Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Thu, 8 Aug 2024 13:02:43 -0500 Subject: [PATCH 04/24] checkpoint; test fixes move prioritizer overwriting into fixture cleanup backend mocks --- .../_core/launcher/dragon/dragonBackend.py | 67 ++--- tests/test_dragon_run_request.py | 243 +++++++----------- tests/test_mocks.py | 65 +++++ 3 files changed, 181 insertions(+), 194 deletions(-) create mode 100644 tests/test_mocks.py diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index 19e4b6fe6..9b90665b1 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -330,17 +330,22 @@ def _can_honor_policy( an optional error message""" # ensure the policy can be honored if request.policy: + logger.debug(f"{request.policy=}") + logger.debug(f"{self._cpus=}") + logger.debug(f"{self._gpus=}") + if request.policy.cpu_affinity: # make sure some node has enough CPUs - available = max(self._cpus) + last_available = max(self._cpus or [-1]) requested = max(request.policy.cpu_affinity) - if requested >= available: + if not any(self._cpus) or requested >= last_available: return False, "Cannot satisfy request, not enough CPUs available" if request.policy.gpu_affinity: # make sure some node has enough GPUs - available = max(self._gpus) + last_available = max(self._gpus or [-1]) requested = max(request.policy.gpu_affinity) - if requested >= available: + if not any(self._gpus) or requested >= last_available: + print(f"failed check w/{self._gpus=}, {requested=}, {last_available=}") return False, "Cannot satisfy request, not enough GPUs available" return True, None @@ -366,38 +371,6 @@ def _can_honor(self, request: DragonRunRequest) -> t.Tuple[bool, t.Optional[str] if not honorable: return False, err - honorable, err = self._can_honor_affinities(request) - if not honorable: - return False, err - - return True, None - - def _can_honor_affinities( - self, request: DragonRunRequest - ) -> t.Tuple[bool, t.Optional[str]]: - """Check if the policy can be honored with resources available - in the allocation. - - :param request: the DragonRunRequest to verify - :returns: Tuple indicating if the request can be honored and - an optional error message""" - if request.policy: - if request.policy.cpu_affinity: - # make sure some node has enough CPUs - available = max(self._cpus) - requested = max(request.policy.cpu_affinity) - - if requested >= available: - return False, "Cannot satisfy request, not enough CPUs available" - - if request.policy.gpu_affinity: - # make sure some node has enough GPUs - available = max(self._gpus) - requested = max(request.policy.gpu_affinity) - - if requested >= available: - return False, "Cannot satisfy request, not enough GPUs available" - return True, None def _can_honor_hosts( @@ -409,26 +382,32 @@ def _can_honor_hosts( :param request: `DragonRunRequest` to validate :returns: Tuple indicating if the request can be honored and an optional error message""" + all_hosts = frozenset(self._hosts) + num_nodes = request.nodes + # fail if requesting more nodes than the total number available - if request.nodes > len(self._hosts): - message = f"Cannot satisfy request. {request.nodes} requested nodes" - message += f"exceeds {len(self._hosts)} available." + if num_nodes > len(all_hosts): + message = f"Cannot satisfy request. {num_nodes} requested nodes" + message += f"exceeds {len(all_hosts)} available." return False, message - requested_hosts: t.Set[str] = set(self._hosts) + requested_hosts = all_hosts if request.hostlist: - requested_hosts = {host.strip() for host in request.hostlist.split(",")} + requested_hosts = frozenset({host.strip() for host in request.hostlist.split(",")}) - all_hosts = set(self._hosts) valid_hosts = all_hosts.intersection(requested_hosts) invalid_hosts = requested_hosts - valid_hosts + logger.debug(f"{num_nodes=}") + logger.debug(f"{valid_hosts=}") + logger.debug(f"{invalid_hosts=}") + if invalid_hosts: logger.warning(f"Some invalid hostnames were requested: {invalid_hosts}") # fail if requesting specific hostnames and there aren't enough available - if request.nodes > len(valid_hosts): - message = f"Cannot satisfy request. Requested {request.nodes} nodes, " + if num_nodes > len(valid_hosts): + message = f"Cannot satisfy request. Requested {num_nodes} nodes, " message += f"but only {len(valid_hosts)} named hosts are available." return False, message diff --git a/tests/test_dragon_run_request.py b/tests/test_dragon_run_request.py index 4545517ec..627089d5c 100644 --- a/tests/test_dragon_run_request.py +++ b/tests/test_dragon_run_request.py @@ -37,13 +37,7 @@ # The tests in this file belong to the group_b group pytestmark = pytest.mark.group_b - -try: - import dragon - - dragon_loaded = True -except: - dragon_loaded = False +dragon = pytest.importorskip("dragon") from smartsim._core.config import CONFIG from smartsim._core.schemas.dragonRequests import * @@ -58,38 +52,6 @@ ) -class NodeMock(MagicMock): - def __init__( - self, name: t.Optional[str] = None, num_gpus: int = 2, num_cpus: int = 8 - ) -> None: - super().__init__() - self._mock_id = name - NodeMock._num_gpus = num_gpus - NodeMock._num_cpus = num_cpus - - @property - def hostname(self) -> str: - if self._mock_id: - return self._mock_id - return create_short_id_str() - - @property - def num_cpus(self) -> str: - return NodeMock._num_cpus - - @property - def num_gpus(self) -> str: - return NodeMock._num_gpus - - def _set_id(self, value: str) -> None: - self._mock_id = value - - def gpus(self, parent: t.Any = None) -> t.List[str]: - if self._num_gpus: - return [f"{self.hostname}-gpu{i}" for i in range(NodeMock._num_gpus)] - return [] - - class GroupStateMock(MagicMock): def Running(self) -> MagicMock: running = MagicMock(**{"__str__.return_value": "Running"}) @@ -104,71 +66,59 @@ class ProcessGroupMock(MagicMock): puids = [121, 122] -def node_mock() -> NodeMock: - return NodeMock() - - def get_mock_backend( - monkeypatch: pytest.MonkeyPatch, num_gpus: int = 2 + monkeypatch: pytest.MonkeyPatch, num_cpus: int, num_gpus: int ) -> "DragonBackend": - + # create all the necessary namespaces as raw magic mocks + monkeypatch.setitem(sys.modules, "dragon.data.ddict.ddict", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.native.machine", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.native.group_state", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.native.process_group", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.native.process", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.infrastructure.connection", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.infrastructure.policy", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.infrastructure.process_desc", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.data.ddict.ddict", MagicMock()) + + node_list = ["node1", "node2", "node3"] + system_mock = MagicMock(return_value=MagicMock(nodes=node_list)) + node_mock = lambda x: MagicMock(hostname=x, num_cpus=num_cpus, num_gpus=num_gpus) + process_group_mock = MagicMock(return_value=ProcessGroupMock()) process_mock = MagicMock(returncode=0) - process_group_mock = MagicMock(**{"Process.return_value": ProcessGroupMock()}) - process_module_mock = MagicMock() - process_module_mock.Process = process_mock - node_mock = NodeMock(num_gpus=num_gpus) - system_mock = MagicMock(nodes=["node1", "node2", "node3"]) + policy_mock = MagicMock(return_value=MagicMock()) + group_state_mock = GroupStateMock() + + # customize members that must perform specific actions within the namespaces monkeypatch.setitem( sys.modules, "dragon", MagicMock( **{ - "native.machine.Node.return_value": node_mock, - "native.machine.System.return_value": system_mock, - "native.group_state": GroupStateMock(), - "native.process_group.ProcessGroup.return_value": ProcessGroupMock(), + "native.machine.Node": node_mock, + "native.machine.System": system_mock, + "native.group_state": group_state_mock, + "native.process_group.ProcessGroup": process_group_mock, + "native.process_group.Process": process_mock, + "native.process.Process": process_mock, + "infrastructure.policy.Policy": policy_mock, } ), ) - monkeypatch.setitem( - sys.modules, - "dragon.infrastructure.connection", - MagicMock(), - ) - monkeypatch.setitem( - sys.modules, - "dragon.infrastructure.process_desc", - MagicMock(), - ) - monkeypatch.setitem( - sys.modules, - "dragon.data.ddict.ddict", - MagicMock(), - ) - monkeypatch.setitem( - sys.modules, - "dragon.infrastructure.policy", - MagicMock(**{"Policy.return_value": MagicMock()}), - ) - monkeypatch.setitem(sys.modules, "dragon.native.process", process_module_mock) - monkeypatch.setitem(sys.modules, "dragon.native.process_group", process_group_mock) - monkeypatch.setitem(sys.modules, "dragon.native.group_state", GroupStateMock()) - monkeypatch.setitem( - sys.modules, - "dragon.native.machine", - MagicMock( - **{"System.return_value": system_mock, "Node.return_value": node_mock} - ), - ) from smartsim._core.launcher.dragon.dragonBackend import DragonBackend dragon_backend = DragonBackend(pid=99999) - # monkeypatch.setattr( - # dragon_backend._prioritizer, - # NodePrioritizer(dragon_backend._hosts, dragon_backend._queue_lock), - # collections.deque(dragon_backend._hosts), - # ) + + # NOTE: we're manually updating these values due to issue w/mocking namespaces + dragon_backend._prioritizer = NodePrioritizer( + [ + MagicMock(num_cpus=num_cpus, num_gpus=num_gpus, hostname=node) + for node in node_list + ], + dragon_backend._queue_lock, + ) + dragon_backend._cpus = [num_cpus] * len(node_list) + dragon_backend._gpus = [num_gpus] * len(node_list) return dragon_backend @@ -232,9 +182,8 @@ def set_mock_group_infos( return group_infos -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_handshake_request(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) handshake_req = DragonHandshakeRequest() handshake_resp = dragon_backend.process_request(handshake_req) @@ -243,9 +192,8 @@ def test_handshake_request(monkeypatch: pytest.MonkeyPatch) -> None: assert handshake_resp.dragon_pid == 99999 -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_run_request(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -287,8 +235,6 @@ def test_run_request(monkeypatch: pytest.MonkeyPatch) -> None: assert len(dragon_backend.free_hosts) == 1 assert dragon_backend._allocated_hosts[dragon_backend.hosts[0]] == step_id assert dragon_backend._allocated_hosts[dragon_backend.hosts[1]] == step_id - # assert dragon_backend._assigned_steps[dragon_backend.hosts[0]] == step_id - # assert dragon_backend._assigned_steps[dragon_backend.hosts[1]] == step_id dragon_backend._group_infos[step_id].status = SmartSimStatus.STATUS_CANCELLED @@ -296,9 +242,8 @@ def test_run_request(monkeypatch: pytest.MonkeyPatch) -> None: assert not dragon_backend._running_steps -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_deny_run_request(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) dragon_backend._shutdown_requested = True @@ -324,7 +269,7 @@ def test_deny_run_request(monkeypatch: pytest.MonkeyPatch) -> None: def test_run_request_with_empty_policy(monkeypatch: pytest.MonkeyPatch) -> None: """Verify that a policy is applied to a run request""" - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -340,10 +285,9 @@ def test_run_request_with_empty_policy(monkeypatch: pytest.MonkeyPatch) -> None: assert run_req.policy is None -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_run_request_with_policy(monkeypatch: pytest.MonkeyPatch) -> None: """Verify that a policy is applied to a run request""" - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -383,7 +327,7 @@ def test_run_request_with_policy(monkeypatch: pytest.MonkeyPatch) -> None: assert dragon_backend._running_steps == [step_id] assert len(dragon_backend._queued_steps) == 0 - assert len(dragon_backend._priorizer.unassigned()) == 1 + assert len(dragon_backend._prioritizer.unassigned()) == 1 assert dragon_backend._allocated_hosts[dragon_backend.hosts[0]] == step_id assert dragon_backend._allocated_hosts[dragon_backend.hosts[1]] == step_id @@ -393,9 +337,8 @@ def test_run_request_with_policy(monkeypatch: pytest.MonkeyPatch) -> None: assert not dragon_backend._running_steps -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_udpate_status_request(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) group_infos = set_mock_group_infos(monkeypatch, dragon_backend) @@ -410,9 +353,8 @@ def test_udpate_status_request(monkeypatch: pytest.MonkeyPatch) -> None: } -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_stop_request(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) group_infos = set_mock_group_infos(monkeypatch, dragon_backend) running_steps = [ @@ -442,7 +384,6 @@ def test_stop_request(monkeypatch: pytest.MonkeyPatch) -> None: assert len(dragon_backend._prioritizer.unassigned()) == 3 -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @pytest.mark.parametrize( "immediate, kill_jobs, frontend_shutdown", [ @@ -461,7 +402,7 @@ def test_shutdown_request( frontend_shutdown: bool, ) -> None: monkeypatch.setenv("SMARTSIM_FLAG_TELEMETRY", "0") - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) monkeypatch.setattr(dragon_backend, "_cooldown_period", 1) set_mock_group_infos(monkeypatch, dragon_backend) @@ -501,11 +442,10 @@ def test_shutdown_request( assert dragon_backend._has_cooled_down == kill_jobs -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @pytest.mark.parametrize("telemetry_flag", ["0", "1"]) def test_cooldown_is_set(monkeypatch: pytest.MonkeyPatch, telemetry_flag: str) -> None: monkeypatch.setenv("SMARTSIM_FLAG_TELEMETRY", telemetry_flag) - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) expected_cooldown = ( 2 * CONFIG.telemetry_frequency + 5 if int(telemetry_flag) > 0 else 5 @@ -517,19 +457,17 @@ def test_cooldown_is_set(monkeypatch: pytest.MonkeyPatch, telemetry_flag: str) - assert dragon_backend.cooldown_period == expected_cooldown -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_heartbeat_and_time(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) first_heartbeat = dragon_backend.last_heartbeat assert dragon_backend.current_time > first_heartbeat dragon_backend._heartbeat() assert dragon_backend.last_heartbeat > first_heartbeat -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @pytest.mark.parametrize("num_nodes", [1, 3, 100]) def test_can_honor(monkeypatch: pytest.MonkeyPatch, num_nodes: int) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -549,13 +487,12 @@ def test_can_honor(monkeypatch: pytest.MonkeyPatch, num_nodes: int) -> None: assert error_msg is None if nodes_in_range else error_msg is not None -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @pytest.mark.parametrize("num_nodes", [-10, -1, 0]) def test_can_honor_invalid_num_nodes( monkeypatch: pytest.MonkeyPatch, num_nodes: int ) -> None: """Verify that requests for invalid numbers of nodes (negative, zero) are rejected""" - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) with pytest.raises(pydantic.error_wrappers.ValidationError) as ex: DragonRunRequest( @@ -571,13 +508,14 @@ def test_can_honor_invalid_num_nodes( ) -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @pytest.mark.parametrize("affinity", [[0], [0, 1], list(range(8))]) def test_can_honor_cpu_affinity( monkeypatch: pytest.MonkeyPatch, affinity: t.List[int] ) -> None: """Verify that valid CPU affinities are accepted""" - dragon_backend = get_mock_backend(monkeypatch) + num_cpus, num_gpus = 8, 0 + dragon_backend = get_mock_backend(monkeypatch, num_cpus=num_cpus, num_gpus=num_gpus) + run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -594,11 +532,10 @@ def test_can_honor_cpu_affinity( assert dragon_backend._can_honor(run_req)[0] -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_can_honor_cpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> None: """Verify that invalid CPU affinities are NOT accepted NOTE: negative values are captured by the Pydantic schema""" - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -615,13 +552,15 @@ def test_can_honor_cpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> assert not dragon_backend._can_honor(run_req)[0] -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") @pytest.mark.parametrize("affinity", [[0], [0, 1]]) def test_can_honor_gpu_affinity( monkeypatch: pytest.MonkeyPatch, affinity: t.List[int] ) -> None: """Verify that valid GPU affinities are accepted""" - dragon_backend = get_mock_backend(monkeypatch) + + num_cpus, num_gpus = 8, 2 + dragon_backend = get_mock_backend(monkeypatch, num_cpus=num_cpus, num_gpus=num_gpus) + run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -638,11 +577,10 @@ def test_can_honor_gpu_affinity( assert dragon_backend._can_honor(run_req)[0] -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_can_honor_gpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> None: """Verify that invalid GPU affinities are NOT accepted NOTE: negative values are captured by the Pydantic schema""" - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) run_req = DragonRunRequest( exe="sleep", exe_args=["5"], @@ -659,50 +597,49 @@ def test_can_honor_gpu_affinity_out_of_range(monkeypatch: pytest.MonkeyPatch) -> assert not dragon_backend._can_honor(run_req)[0] -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_can_honor_gpu_device_not_available(monkeypatch: pytest.MonkeyPatch) -> None: """Verify that a request for a GPU if none exists is not accepted""" # create a mock node class that always reports no GPUs available - dragon_backend = get_mock_backend(monkeypatch, num_gpus=0) + with monkeypatch.context() as ctx: + dragon_backend = get_mock_backend(ctx, num_cpus=8, num_gpus=0) - run_req = DragonRunRequest( - exe="sleep", - exe_args=["5"], - path="/a/fake/path", - nodes=2, - tasks=1, - tasks_per_node=1, - env={}, - current_env={}, - pmi_enabled=False, - # specify GPU device w/no affinity - policy=DragonRunPolicy(gpu_affinity=[0]), - ) - - assert not dragon_backend._can_honor(run_req)[0] + run_req = DragonRunRequest( + exe="sleep", + exe_args=["5"], + path="/a/fake/path", + nodes=2, + tasks=1, + tasks_per_node=1, + env={}, + current_env={}, + pmi_enabled=False, + # specify GPU device w/no affinity + policy=DragonRunPolicy(gpu_affinity=[0]), + ) + can_honor, _ = dragon_backend._can_honor(run_req) + assert not can_honor -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_get_id(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) step_id = next(dragon_backend._step_ids) assert step_id.endswith("0") assert step_id != next(dragon_backend._step_ids) -@pytest.mark.skipif(not dragon_loaded, reason="Test is only for Dragon WLM systems") def test_view(monkeypatch: pytest.MonkeyPatch) -> None: - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) set_mock_group_infos(monkeypatch, dragon_backend) hosts = dragon_backend.hosts + dragon_backend._prioritizer.increment(hosts[0]) - expected_message = textwrap.dedent( + expected_msg = textwrap.dedent( f"""\ Dragon server backend update | Host | Status | - |---------|----------| + |--------|----------| | {hosts[0]} | Busy | | {hosts[1]} | Free | | {hosts[2]} | Free | @@ -717,15 +654,21 @@ def test_view(monkeypatch: pytest.MonkeyPatch) -> None: # get rid of white space to make the comparison easier actual_msg = dragon_backend.status_message.replace(" ", "") - expected_message = expected_message.replace(" ", "") + expected_msg = expected_msg.replace(" ", "") + + # ignore dashes in separators (hostname changes may cause column expansion) + while actual_msg.find("--") > -1: + actual_msg = actual_msg.replace("--", "-") + while expected_msg.find("--") > -1: + expected_msg = expected_msg.replace("--", "-") - assert actual_msg == expected_message + assert actual_msg == expected_msg def test_can_honor_hosts_unavailable_hosts(monkeypatch: pytest.MonkeyPatch) -> None: """Verify that requesting nodes with invalid names causes number of available nodes check to fail due to valid # of named nodes being under num_nodes""" - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) # let's supply 2 invalid and 1 valid hostname actual_hosts = list(dragon_backend._hosts) @@ -759,7 +702,7 @@ def test_can_honor_hosts_unavailable_hosts(monkeypatch: pytest.MonkeyPatch) -> N def test_can_honor_hosts_unavailable_hosts_ok(monkeypatch: pytest.MonkeyPatch) -> None: """Verify that requesting nodes with invalid names causes number of available nodes check to be reduced but still passes if enough valid named nodes are passed""" - dragon_backend = get_mock_backend(monkeypatch) + dragon_backend = get_mock_backend(monkeypatch, num_cpus=8, num_gpus=0) # let's supply 2 valid and 1 invalid hostname actual_hosts = list(dragon_backend._hosts) diff --git a/tests/test_mocks.py b/tests/test_mocks.py new file mode 100644 index 000000000..227318a2d --- /dev/null +++ b/tests/test_mocks.py @@ -0,0 +1,65 @@ +from unittest.mock import MagicMock + +import pytest +import sys +import uuid + + +class ProcessGroupMock(MagicMock): + puids = [121, 122] + + +def test_mocked_backend(monkeypatch: pytest.MonkeyPatch): + system_mock = MagicMock(nodes=["node1", "node2", "node3"]) + process_group_mock = ProcessGroupMock() + + # node_mock = lambda: MagicMock(num_cpus=4, num_gpus=2, ident=str(uuid.uuid4())) + + monkeypatch.setitem( + sys.modules, + "dragon", + MagicMock(**{"data.ddict.ddict": MagicMock(**{"DDict": MagicMock()})}), + ) + + # monkeypatch.setitem( + # sys.modules, "dragon.data.ddict.ddict", MagicMock(**{"DDict": MagicMock()}) + # ) + monkeypatch.setitem(sys.modules, "dragon.infrastructure.connection", MagicMock()) + monkeypatch.setitem( + sys.modules, + "dragon.infrastructure.policy", + MagicMock(**{"Policy.return_value": MagicMock()}), + ) + monkeypatch.setitem(sys.modules, "dragon.infrastructure.process_desc", MagicMock()) + monkeypatch.setitem(sys.modules, "dragon.native.group_state", MagicMock()) + monkeypatch.setitem( + sys.modules, + "dragon.native.machine", + MagicMock( + **{ + "System.return_value": system_mock, + "Node.return_value": MagicMock( + num_cpus=4, num_gpus=2, ident=str(uuid.uuid4()) + ), + } + ), + ) + monkeypatch.setitem( + sys.modules, + "dragon.native.process", + MagicMock(**{"Process": MagicMock(returncode=0)}), + ) + monkeypatch.setitem( + sys.modules, + "dragon.native.process_group", + MagicMock(**{"Process.return_value": process_group_mock}), + ) + + import dragon.data.ddict.ddict as dragon_dict + from smartsim._core.launcher.dragon.dragonBackend import DragonBackend + + dd = dragon_dict.DDict() + dd["foo"] = "bar" + + backend = DragonBackend(42) + assert backend._pid == 42 From 35fa202a89ab35bb68aa03cb5c060d70c0de0d3c Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Fri, 9 Aug 2024 11:00:26 -0500 Subject: [PATCH 05/24] formatting --- .../_core/launcher/dragon/dragonBackend.py | 8 ++- smartsim/_core/launcher/step/dragonStep.py | 2 +- tests/test_dragon_run_request.py | 6 +- tests/test_mocks.py | 65 ------------------- 4 files changed, 9 insertions(+), 72 deletions(-) delete mode 100644 tests/test_mocks.py diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index 9b90665b1..a85ba7aa5 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -345,7 +345,9 @@ def _can_honor_policy( last_available = max(self._gpus or [-1]) requested = max(request.policy.gpu_affinity) if not any(self._gpus) or requested >= last_available: - print(f"failed check w/{self._gpus=}, {requested=}, {last_available=}") + print( + f"failed check w/{self._gpus=}, {requested=}, {last_available=}" + ) return False, "Cannot satisfy request, not enough GPUs available" return True, None @@ -393,7 +395,9 @@ def _can_honor_hosts( requested_hosts = all_hosts if request.hostlist: - requested_hosts = frozenset({host.strip() for host in request.hostlist.split(",")}) + requested_hosts = frozenset( + {host.strip() for host in request.hostlist.split(",")} + ) valid_hosts = all_hosts.intersection(requested_hosts) invalid_hosts = requested_hosts - valid_hosts diff --git a/smartsim/_core/launcher/step/dragonStep.py b/smartsim/_core/launcher/step/dragonStep.py index bca693ce7..cf0cc3983 100644 --- a/smartsim/_core/launcher/step/dragonStep.py +++ b/smartsim/_core/launcher/step/dragonStep.py @@ -31,8 +31,8 @@ import typing as t from ...._core.schemas.dragonRequests import ( - DragonRunRequest, DragonRunPolicy, + DragonRunRequest, request_registry, ) from ....error.errors import SSUnsupportedError diff --git a/tests/test_dragon_run_request.py b/tests/test_dragon_run_request.py index 627089d5c..5ff95f408 100644 --- a/tests/test_dragon_run_request.py +++ b/tests/test_dragon_run_request.py @@ -635,8 +635,7 @@ def test_view(monkeypatch: pytest.MonkeyPatch) -> None: hosts = dragon_backend.hosts dragon_backend._prioritizer.increment(hosts[0]) - expected_msg = textwrap.dedent( - f"""\ + expected_msg = textwrap.dedent(f"""\ Dragon server backend update | Host | Status | |--------|----------| @@ -649,8 +648,7 @@ def test_view(monkeypatch: pytest.MonkeyPatch) -> None: | del999-2 | Cancelled | {hosts[1]} | -9 | 1 | | c101vz-3 | Completed | {hosts[1]},{hosts[2]} | 0 | 2 | | 0ghjk1-4 | Failed | {hosts[2]} | -1 | 1 | - | ljace0-5 | NeverStarted | | | 0 |""" - ) + | ljace0-5 | NeverStarted | | | 0 |""") # get rid of white space to make the comparison easier actual_msg = dragon_backend.status_message.replace(" ", "") diff --git a/tests/test_mocks.py b/tests/test_mocks.py deleted file mode 100644 index 227318a2d..000000000 --- a/tests/test_mocks.py +++ /dev/null @@ -1,65 +0,0 @@ -from unittest.mock import MagicMock - -import pytest -import sys -import uuid - - -class ProcessGroupMock(MagicMock): - puids = [121, 122] - - -def test_mocked_backend(monkeypatch: pytest.MonkeyPatch): - system_mock = MagicMock(nodes=["node1", "node2", "node3"]) - process_group_mock = ProcessGroupMock() - - # node_mock = lambda: MagicMock(num_cpus=4, num_gpus=2, ident=str(uuid.uuid4())) - - monkeypatch.setitem( - sys.modules, - "dragon", - MagicMock(**{"data.ddict.ddict": MagicMock(**{"DDict": MagicMock()})}), - ) - - # monkeypatch.setitem( - # sys.modules, "dragon.data.ddict.ddict", MagicMock(**{"DDict": MagicMock()}) - # ) - monkeypatch.setitem(sys.modules, "dragon.infrastructure.connection", MagicMock()) - monkeypatch.setitem( - sys.modules, - "dragon.infrastructure.policy", - MagicMock(**{"Policy.return_value": MagicMock()}), - ) - monkeypatch.setitem(sys.modules, "dragon.infrastructure.process_desc", MagicMock()) - monkeypatch.setitem(sys.modules, "dragon.native.group_state", MagicMock()) - monkeypatch.setitem( - sys.modules, - "dragon.native.machine", - MagicMock( - **{ - "System.return_value": system_mock, - "Node.return_value": MagicMock( - num_cpus=4, num_gpus=2, ident=str(uuid.uuid4()) - ), - } - ), - ) - monkeypatch.setitem( - sys.modules, - "dragon.native.process", - MagicMock(**{"Process": MagicMock(returncode=0)}), - ) - monkeypatch.setitem( - sys.modules, - "dragon.native.process_group", - MagicMock(**{"Process.return_value": process_group_mock}), - ) - - import dragon.data.ddict.ddict as dragon_dict - from smartsim._core.launcher.dragon.dragonBackend import DragonBackend - - dd = dragon_dict.DDict() - dd["foo"] = "bar" - - backend = DragonBackend(42) - assert backend._pid == 42 From 1aac4e190b21ed2668bd81a29ad760a97d4ba9f4 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Fri, 9 Aug 2024 11:14:25 -0500 Subject: [PATCH 06/24] remove problematic import --- tests/dragon/test_worker_manager.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/dragon/test_worker_manager.py b/tests/dragon/test_worker_manager.py index d6f5c2ac0..c8332c260 100644 --- a/tests/dragon/test_worker_manager.py +++ b/tests/dragon/test_worker_manager.py @@ -32,8 +32,6 @@ import pytest -from tests.mli.featurestore import FileSystemFeatureStore - torch = pytest.importorskip("torch") dragon = pytest.importorskip("dragon") From 774b15972b5a18c0b0857c9f6ba038b8978b3cb6 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Fri, 9 Aug 2024 12:35:40 -0500 Subject: [PATCH 07/24] reset me and add the tests to fix break --- smartsim/settings/dragonRunSettings.py | 19 ++++++++++++++++++- tests/test_dragon_step.py | 13 +++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/smartsim/settings/dragonRunSettings.py b/smartsim/settings/dragonRunSettings.py index 61434b70d..559c0fb88 100644 --- a/smartsim/settings/dragonRunSettings.py +++ b/smartsim/settings/dragonRunSettings.py @@ -101,7 +101,24 @@ def set_hostlist(self, host_list: t.Union[str, t.List[str]]) -> None: :param host_list: hosts to launch on """ - self.run_args["host-list"] = ",".join(host_list) + if not host_list: + raise + if isinstance(host_list, list): + self.run_args["host-list"] = ",".join(host_list.replace(" ", "")) + return + + BREAKING ON PURPOSE! + Add a test for DragonRunSettings for: + - set_hotslist (None) + - set_hostlist([]) + - set_hostlist(['']) + - set_hostlist(['abc']) - single + - set_hostlist(['abc', 'def']) - multi + - set_hostlist('abc') - single (str) + - set_hostlist('abc,def']) - multi (str), + - set_hostlist('abc, def']) - multi (str), spaces removal 'x, y' -> 'x','y', not 'x', ' y' + + self.run_args["host-list"] = host_list.replace(" ", "") def set_cpu_affinity(self, devices: t.List[int]) -> None: """Set the CPU affinity for this job diff --git a/tests/test_dragon_step.py b/tests/test_dragon_step.py index 19f408e0b..f933fb7bc 100644 --- a/tests/test_dragon_step.py +++ b/tests/test_dragon_step.py @@ -73,12 +73,18 @@ def dragon_batch_step(test_dir: str) -> DragonBatchStep: cpu_affinities = [[], [0, 1, 2], [], [3, 4, 5, 6]] gpu_affinities = [[], [], [0, 1, 2], [3, 4, 5, 6]] + # specify 3 hostnames to select from but require only 2 nodes + num_nodes = 2 + hostnames = ["host1", "host2", "host3"] + # assign some unique affinities to each run setting instance for index, rs in enumerate(settings): if gpu_affinities[index]: rs.set_node_feature("gpu") rs.set_cpu_affinity(cpu_affinities[index]) rs.set_gpu_affinity(gpu_affinities[index]) + rs.set_hostlist(hostnames) + rs.set_nodes(num_nodes) steps = list( DragonStep(name_, test_dir, rs_) for name_, rs_ in zip(names, settings) @@ -374,6 +380,11 @@ def test_dragon_batch_step_write_request_file( cpu_affinities = [[], [0, 1, 2], [], [3, 4, 5, 6]] gpu_affinities = [[], [], [0, 1, 2], [3, 4, 5, 6]] + hostnames = ["host1", "host2", "host3"] + num_nodes = 2 + + # parse requests file path from the launch command + # e.g. dragon python launch_cmd = dragon_batch_step.get_launch_cmd() requests_file = get_request_path_from_batch_script(launch_cmd) @@ -392,3 +403,5 @@ def test_dragon_batch_step_write_request_file( assert run_request assert run_request.policy.cpu_affinity == cpu_affinities[index] assert run_request.policy.gpu_affinity == gpu_affinities[index] + assert run_request.nodes == num_nodes + assert run_request.hostlist == ",".join(hostnames) From e467673fd0fafa313c25ecb330f47e02a230d36b Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Fri, 9 Aug 2024 18:17:04 -0500 Subject: [PATCH 08/24] refactor prioritizer to avoid list/typedef --- .../_core/launcher/dragon/dragonBackend.py | 15 +- smartsim/_core/launcher/dragon/pqueue.py | 208 ++++++++++++------ smartsim/settings/dragonRunSettings.py | 30 +-- tests/test_node_prioritizer.py | 108 +++++---- 4 files changed, 224 insertions(+), 137 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index a85ba7aa5..0e8a5fdaa 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -218,7 +218,7 @@ def free_hosts(self) -> t.List[str]: :returns: List of host names""" with self._queue_lock: - return [str(item[1]) for item in self._prioritizer.unassigned()] + return [node.hostname for node in self._prioritizer.unassigned()] @property def group_infos(self) -> dict[str, ProcessGroupInfo]: @@ -456,22 +456,21 @@ def _allocate_step( hosts = [host for host in request.hostlist.split(",") if host] if hosts: - reference_counts = self._prioritizer.next_n_from(num_hosts, hosts) + nodes = self._prioritizer.next_n_from(num_hosts, hosts) else: filter_on: t.Optional[PrioritizerFilter] = None if request.policy and request.policy.gpu_affinity: filter_on = PrioritizerFilter.GPU - reference_counts = self._prioritizer.next_n(num_hosts, filter_on) + nodes = self._prioritizer.next_n(num_hosts, filter_on) - if len(reference_counts) < num_hosts: + if len(nodes) < num_hosts: # exit if the prioritizer can't identify enough nodes return None - to_allocate = [str(ref_counter[1]) for ref_counter in reference_counts] - + to_allocate = [node.hostname for node in nodes] # track assigning this step to each node - for host in to_allocate: - self._allocated_hosts[host] = step_id + for hostname in to_allocate: + self._allocated_hosts[hostname] = step_id return to_allocate diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index 3b330b597..ba036d92d 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -23,10 +23,12 @@ # CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, # OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE # OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +# import collections import enum import heapq import threading import typing as t +from dataclasses import dataclass, field from smartsim.error.errors import SmartSimError from smartsim.log import get_logger @@ -34,8 +36,68 @@ logger = get_logger(__name__) -# tracking structure for [num_refs, node_name, is_dirty] -_NodeRefCount = t.List[t.Union[int, str]] +@dataclass +class _TrackedNode: + """Minimum Node API required to support the NodePrioritizer""" + + num_cpus: int + """The number of CPUs available on this node""" + num_gpus: int + """The number of GPUs available on this node""" + hostname: str + """The hostname of this node""" + num_refs: int = 0 + """The number of processes currently using this node""" + + tracking: t.Set[str] = field(default_factory=set) + """The unique identifiers of processes using this node""" + allocated_cpus: t.Set[int] = field(default_factory=set) + """The CPU indices allocated on this node""" + allocated_gpus: t.Set[int] = field(default_factory=set) + """The GPU indices allocated on this node""" + dirty: bool = False + """Flag indicating that the node has been updated""" + + def add( + self, + tracking_id: t.Optional[str] = None, + cpus: t.Optional[t.Sequence[int]] = None, + gpus: t.Optional[t.Sequence[int]] = None, + ) -> None: + if tracking_id in self.tracking: + raise ValueError("Attempted adding task more than once") + + self.num_refs = self.num_refs + 1 + if tracking_id: + self.tracking = self.tracking.union({tracking_id}) + if cpus: + self.allocated_cpus.update(set(cpus)) + if gpus: + self.allocated_gpus.update(set(gpus)) + self.dirty = True + + def remove( + self, + tracking_id: t.Optional[str] = None, + cpus: t.Optional[t.Sequence[int]] = None, + gpus: t.Optional[t.Sequence[int]] = None, + ) -> None: + if tracking_id and tracking_id not in self.tracking: + raise ValueError("Attempted removal of untracked item") + + self.num_refs = max(self.num_refs - 1, 0) + self.tracking = self.tracking - {tracking_id} + if cpus: + self.allocated_cpus.difference_update(set(cpus)) + if gpus: + self.allocated_gpus.difference_update(set(gpus)) + self.dirty = True + + def __lt__(self, other: "_TrackedNode") -> bool: + if self.num_refs < other.num_refs: + return True + + return False class PrioritizerFilter(str, enum.Enum): @@ -48,14 +110,16 @@ class PrioritizerFilter(str, enum.Enum): class Node(t.Protocol): """Minimum Node API required to support the NodePrioritizer""" - @property - def num_cpus(self) -> int: ... + hostname: str + num_cpus: int + num_gpus: int + - @property - def num_gpus(self) -> int: ... +class NodeReferenceCount(t.Protocol): + """Contains details pertaining to references to a node""" - @property - def hostname(self) -> str: ... + hostname: str + num_refs: int class NodePrioritizer: @@ -71,15 +135,14 @@ def __init__(self, nodes: t.List[Node], lock: threading.RLock) -> None: self._lock = lock """Lock used to ensure thread safe changes of the reference counters""" - self._ref_map: t.Dict[str, _NodeRefCount] = {} - """Map node names to a ref counter for direct access""" - - self._cpu_refs: t.List[_NodeRefCount] = [] + self._cpu_refs: t.List[_TrackedNode] = [] """Track reference counts to CPU-only nodes""" - self._gpu_refs: t.List[_NodeRefCount] = [] + self._gpu_refs: t.List[_TrackedNode] = [] """Track reference counts to GPU nodes""" + self._nodes: t.Dict[str, _TrackedNode] = {} + self._initialize_reference_counters(nodes) def _initialize_reference_counters(self, nodes: t.List[Node]) -> None: @@ -88,31 +151,40 @@ def _initialize_reference_counters(self, nodes: t.List[Node]) -> None: :param nodes: node attribute information for initializing the priorizer""" for node in nodes: # initialize all node counts to 0 and mark the entries "is_dirty=False" - tracking_info: _NodeRefCount = [ - 0, - node.hostname, - 0, - ] # use list for mutability + tracked = _TrackedNode( + node.num_cpus, node.num_gpus, node.hostname, 0, set() + ) - self._ref_map[node.hostname] = tracking_info + self._nodes[node.hostname] = tracked # for O(1) access if node.num_gpus: - self._gpu_refs.append(tracking_info) + self._gpu_refs.append(tracked) else: - self._cpu_refs.append(tracking_info) + self._cpu_refs.append(tracked) + + # def _update_ref_count(self, host: str, updated_ref_count: _TrackedNode) -> None: + # """Updates the shared _NodeRefCount instance to keep each + # reference (cpu ref, gpu ref, all refs) in sync""" + # node = self._nodes[host] + + # node.num_refs = updated_ref_count[0] + # node.dirty = updated_ref_count[2] - def increment(self, host: str) -> None: + def increment( + self, host: str, tracking_id: t.Optional[str] = None + ) -> NodeReferenceCount: """Directly increment the reference count of a given node and ensure the ref counter is marked as dirty to trigger a reordering on retrieval :param host: a hostname that should have a reference counter selected""" with self._lock: - tracking_info = self._ref_map[host] - ref_count, *_ = tracking_info - tracking_info[0] = int(ref_count) + 1 - tracking_info[2] = 1 + tracked_node = self._nodes[host] + tracked_node.add(tracking_id) - def _all_refs(self) -> t.List[_NodeRefCount]: + # self._update_ref_count(host, tracked_node) + return tracked_node + + def _all_refs(self) -> t.List[_TrackedNode]: """Combine the CPU and GPU nodes into a single heap :returns: list of all reference counters""" @@ -120,42 +192,46 @@ def _all_refs(self) -> t.List[_NodeRefCount]: heapq.heapify(refs) return refs - def get_tracking_info(self, host: str) -> _NodeRefCount: + def get_tracking_info(self, host: str) -> NodeReferenceCount: """Returns the reference counter information for a single node :param host: a hostname that should have a reference counter selected :returns: a reference counter""" - return self._ref_map[host] + return self._nodes[host] - def decrement(self, host: str) -> None: + def decrement( + self, host: str, tracking_id: t.Optional[str] = None + ) -> NodeReferenceCount: """Directly increment the reference count of a given node and ensure the ref counter is marked as dirty to trigger a reordering :param host: a hostname that should have a reference counter decremented""" with self._lock: - tracking_info = self._ref_map[host] - tracking_info[0] = max(int(tracking_info[0]) - 1, 0) - tracking_info[2] = 1 + tracked_node = self._nodes[host] + tracked_node.remove(tracking_id) + + # self._update_ref_count(host, tracked_node.as_refcount()) + return tracked_node - def _create_sub_heap(self, hosts: t.List[str]) -> t.List[_NodeRefCount]: + def _create_sub_heap(self, hosts: t.List[str]) -> t.List[_TrackedNode]: """Create a new heap from the primary heap with user-specified nodes :param hosts: a list of hostnames used to filter the available nodes :returns: a list of assigned reference counters """ - nodes_tracking_info: t.List[_NodeRefCount] = [] + nodes_tracking_info: t.List[_TrackedNode] = [] # Collect all the tracking info for the requested nodes... for host in hosts: - tracking_info = self._ref_map[host] - nodes_tracking_info.append(tracking_info) + if tracking_info := self._nodes.get(host, None): + nodes_tracking_info.append(tracking_info) # ... and use it to create a new heap from a specified subset of nodes heapq.heapify(nodes_tracking_info) return nodes_tracking_info - def next_from(self, hosts: t.List[str]) -> t.Optional[_NodeRefCount]: + def next_from(self, hosts: t.List[str]) -> t.Optional[Node]: """Return the next node available given a set of desired hosts :param hosts: a list of hostnames used to filter the available nodes @@ -167,7 +243,7 @@ def next_from(self, hosts: t.List[str]) -> t.Optional[_NodeRefCount]: sub_heap = self._create_sub_heap(hosts) return self._get_next_available_node(sub_heap) - def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.List[_NodeRefCount]: + def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.List[Node]: """Return the next N available nodes given a set of desired hosts :param num_items: the desird number of nodes to allocate @@ -183,39 +259,36 @@ def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.List[_NodeRefCoun sub_heap = self._create_sub_heap(hosts) return self._get_next_n_available_nodes(num_items, sub_heap) - def unassigned( - self, heap: t.Optional[t.List[_NodeRefCount]] = None - ) -> t.List[_NodeRefCount]: + def unassigned(self, heap: t.Optional[t.List[_TrackedNode]] = None) -> t.List[Node]: """Select nodes that are currently not assigned a task :param heap: (optional) a subset of the node heap to consider :returns: a list of reference counts for all unassigned nodes""" if heap is None: - return [node for node in self._ref_map.values() if node[0] == 0] + heap = list(self._nodes.values()) - return [node for node in heap if node[0] == 0] + return list(filter(lambda x: x.num_refs == 0, heap)) - def assigned( - self, heap: t.Optional[t.List[_NodeRefCount]] = None - ) -> t.List[_NodeRefCount]: + def assigned(self, heap: t.Optional[t.List[_TrackedNode]] = None) -> t.List[Node]: """Helper method to identify the nodes that are currently assigned :param heap: (optional) a subset of the node heap to consider :returns: a list of reference counts for all assigned nodes""" if heap is None: - return [node for node in self._ref_map.values() if node[0] == 1] + heap = list(self._nodes.values()) - return [node for node in heap if node[0] == 1] + return list(filter(lambda x: x.num_refs == 1, heap)) def _check_satisfiable_n( - self, num_items: int, heap: t.Optional[t.List[_NodeRefCount]] = None + self, num_items: int, heap: t.Optional[t.List[_TrackedNode]] = None ) -> bool: """Validates that a request for some number of nodes `n` can be satisfied by the prioritizer given the set of nodes available :param num_items: the desird number of nodes to allocate - :param heap: (optional) a subset of the node heap to consider""" - num_nodes = len(self._ref_map.keys()) + :param heap: (optional) a subset of the node heap to consider + :returns: True if the request can be fulfilled, False otherwise""" + num_nodes = len(self._nodes.keys()) if num_items < 1: msg = "Cannot handle request; request requires a positive integer" @@ -235,9 +308,7 @@ def _check_satisfiable_n( return True - def _get_next_available_node( - self, heap: t.List[_NodeRefCount] - ) -> t.Optional[_NodeRefCount]: + def _get_next_available_node(self, heap: t.List[_TrackedNode]) -> t.Optional[Node]: """Finds the next node w/the least amount of running processes and ensures that any elements that were directly updated are updated in the priority structure before being made available @@ -245,25 +316,25 @@ def _get_next_available_node( :param heap: (optional) a subset of the node heap to consider :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" - tracking_info: t.Optional[_NodeRefCount] = None + tracking_info: t.Optional[_TrackedNode] = None with self._lock: tracking_info = heapq.heappop(heap) - is_dirty = tracking_info[2] + is_dirty = tracking_info.dirty while is_dirty: if is_dirty: # mark dirty items clean and place back into heap to be sorted - tracking_info[2] = 0 + tracking_info.dirty = False heapq.heappush(heap, tracking_info) tracking_info = heapq.heappop(heap) - is_dirty = tracking_info[2] + is_dirty = tracking_info.dirty - original_ref_count = int(tracking_info[0]) + original_ref_count = tracking_info.num_refs if original_ref_count == 0: # increment the ref count before putting back onto heap - tracking_info[0] = original_ref_count + 1 + tracking_info.num_refs += 1 heapq.heappush(heap, tracking_info) @@ -271,13 +342,16 @@ def _get_next_available_node( if original_ref_count > 0: return None + if not tracking_info: + return None + return tracking_info def _get_next_n_available_nodes( self, num_items: int, - heap: t.List[_NodeRefCount], - ) -> t.List[_NodeRefCount]: + heap: t.List[_TrackedNode], + ) -> t.List[Node]: """Find the next N available nodes w/least amount of references using the supplied filter to target a specific node capability @@ -286,7 +360,7 @@ def _get_next_n_available_nodes( unassigned nodes exists, `None` otherwise :raises ValueError: if the number of requetsed nodes is not a positive integer """ - next_nodes: t.List[_NodeRefCount] = [] + next_nodes: t.List[Node] = [] if num_items < 1: raise ValueError(f"Number of items requested {num_items} is invalid") @@ -303,9 +377,7 @@ def _get_next_n_available_nodes( return next_nodes - def next( - self, filter_on: t.Optional[PrioritizerFilter] = None - ) -> t.Optional[_NodeRefCount]: + def next(self, filter_on: t.Optional[PrioritizerFilter] = None) -> t.Optional[Node]: """Find the next node available w/least amount of references using the supplied filter to target a specific node capability @@ -326,7 +398,7 @@ def next( def next_n( self, num_items: int = 1, filter_on: t.Optional[PrioritizerFilter] = None - ) -> t.List[_NodeRefCount]: + ) -> t.List[Node]: """Find the next N available nodes w/least amount of references using the supplied filter to target a specific node capability diff --git a/smartsim/settings/dragonRunSettings.py b/smartsim/settings/dragonRunSettings.py index 559c0fb88..dc8403267 100644 --- a/smartsim/settings/dragonRunSettings.py +++ b/smartsim/settings/dragonRunSettings.py @@ -102,21 +102,25 @@ def set_hostlist(self, host_list: t.Union[str, t.List[str]]) -> None: :param host_list: hosts to launch on """ if not host_list: - raise + raise ValueError("empty hostlist provided") + if isinstance(host_list, list): - self.run_args["host-list"] = ",".join(host_list.replace(" ", "")) + self.run_args["host-list"] = ",".join( + host.replace(" ", "") for host in host_list + ) return - - BREAKING ON PURPOSE! - Add a test for DragonRunSettings for: - - set_hotslist (None) - - set_hostlist([]) - - set_hostlist(['']) - - set_hostlist(['abc']) - single - - set_hostlist(['abc', 'def']) - multi - - set_hostlist('abc') - single (str) - - set_hostlist('abc,def']) - multi (str), - - set_hostlist('abc, def']) - multi (str), spaces removal 'x, y' -> 'x','y', not 'x', ' y' + + # TODO: BREAKING ON PURPOSE! + # Add a test for DragonRunSettings for: + # - set_hotslist (None) + # - set_hostlist([]) + # - set_hostlist(['']) + # - set_hostlist(['abc']) - single + # - set_hostlist(['abc', 'def']) - multi + # - set_hostlist('abc') - single (str) + # - set_hostlist('abc,def']) - multi (str), + # - set_hostlist('abc, def']) - multi (str), + # spaces removal 'x, y' -> 'x','y', not 'x', ' y' self.run_args["host-list"] = host_list.replace(" ", "") diff --git a/tests/test_node_prioritizer.py b/tests/test_node_prioritizer.py index 420308979..2f4a15ea2 100644 --- a/tests/test_node_prioritizer.py +++ b/tests/test_node_prioritizer.py @@ -106,39 +106,39 @@ def test_node_prioritizer_init_ok(num_cpu_nodes: int, num_gpu_nodes: int) -> Non # verify tracking data is initialized correctly for all nodes for hostname in all_hosts: # show that the ref map is tracking the node - assert hostname in p._ref_map + assert hostname in p._nodes - tracking_info = p._ref_map[hostname] + tracking_info = p.get_tracking_info(hostname) # show that the node is created w/zero ref counts - assert tracking_info[0] == 0 + assert tracking_info.num_refs == 0 # show that the node is created and marked as not dirty (unchanged) - assert tracking_info[2] == False + # assert tracking_info.dirty == False # iterate through known cpu node keys and verify prioritizer initialization for hostname in cpu_hosts: # show that the device ref counters are appropriately assigned - cpu_ref = next((n for n in p._cpu_refs if n[1] == hostname), None) + cpu_ref = next((n for n in p._cpu_refs if n.hostname == hostname), None) assert cpu_ref, "CPU-only node not found in cpu ref set" - gpu_ref = next((n for n in p._gpu_refs if n[1] == hostname), None) + gpu_ref = next((n for n in p._gpu_refs if n.hostname == hostname), None) assert not gpu_ref, "CPU-only node should not be found in gpu ref set" # iterate through known GPU node keys and verify prioritizer initialization for hostname in gpu_hosts: # show that the device ref counters are appropriately assigned - gpu_ref = next((n for n in p._gpu_refs if n[1] == hostname), None) + gpu_ref = next((n for n in p._gpu_refs if n.hostname == hostname), None) assert gpu_ref, "GPU-only node not found in gpu ref set" - cpu_ref = next((n for n in p._cpu_refs if n[1] == hostname), None) + cpu_ref = next((n for n in p._cpu_refs if n.hostname == hostname), None) assert not cpu_ref, "GPU-only node should not be found in cpu ref set" # verify we have all hosts in the ref map - assert set(p._ref_map.keys()) == set(all_hosts) + assert set(p._nodes.keys()) == set(all_hosts) # verify we have no extra hosts in ref map - assert len(p._ref_map.keys()) == len(set(all_hosts)) + assert len(p._nodes.keys()) == len(set(all_hosts)) def test_node_prioritizer_direct_increment() -> None: @@ -174,15 +174,15 @@ def test_node_prioritizer_direct_increment() -> None: # ... and verify the correct incrementing is applied tracking_info = p.get_tracking_info(node.hostname) - assert tracking_info[0] == num_increments + assert tracking_info.num_refs == num_increments # verify the excluded cpu node was never changed tracking_info0 = p.get_tracking_info(exclude_host0) - assert tracking_info0[0] == 0 + assert tracking_info0.num_refs == 0 # verify the excluded gpu node was never changed tracking_info1 = p.get_tracking_info(exclude_host1) - assert tracking_info1[0] == 0 + assert tracking_info1.num_refs == 0 def test_node_prioritizer_indirect_increment() -> None: @@ -196,21 +196,28 @@ def test_node_prioritizer_indirect_increment() -> None: lock = threading.RLock() p = NodePrioritizer(nodes, lock) + # verify starting state + for node in p._nodes.values(): + tracking_info = p.get_tracking_info(node.hostname) + + assert node.num_refs == 0 # <--- ref count starts at zero + assert tracking_info.num_refs == 0 # <--- ref count starts at zero + # perform indirect - for i in range(len(nodes)): - tracking_info = p.get_tracking_info(nodes[i].hostname) - assert tracking_info[0] == 0 # <--- ref count starts at zero + for node in p._nodes.values(): + tracking_info = p.get_tracking_info(node.hostname) - # apply operation - tracking_info = p.next(PrioritizerFilter.CPU) + # apply `next` operation and verify tracking info reflects new ref + node = p.next(PrioritizerFilter.CPU) + tracking_info = p.get_tracking_info(node.hostname) # verify side-effects - assert tracking_info[0] > 0 # <--- ref count should now be > 0 - assert ( - tracking_info[2] == False - ) # <--- we expect it to give back only "clean" nodes + assert tracking_info.num_refs > 0 # <--- ref count should now be > 0 + + # we expect it to give back only "clean" nodes from next* + assert tracking_info.dirty == False # NOTE: this is "hidden" by protocol - # every node has been incremented now. prioritizer shouldn't have anything to give + # every node should be incremented now. prioritizer shouldn't have anything to give tracking_info = p.next(PrioritizerFilter.CPU) assert tracking_info is None # <--- get_next shouldn't have any nodes to give @@ -253,25 +260,24 @@ def test_node_prioritizer_multi_increment() -> None: # Mark some nodes as dirty to verify retrieval p.increment(cpu_hosts[0]) - assert p.get_tracking_info(cpu_hosts[0])[0] > 0 + assert p.get_tracking_info(cpu_hosts[0]).num_refs > 0 p.increment(cpu_hosts[2]) - assert p.get_tracking_info(cpu_hosts[2])[0] > 0 + assert p.get_tracking_info(cpu_hosts[2]).num_refs > 0 p.increment(cpu_hosts[4]) - assert p.get_tracking_info(cpu_hosts[4])[0] > 0 + assert p.get_tracking_info(cpu_hosts[4]).num_refs > 0 # use next_n w/the minimum allowed value all_tracking_info = p.next_n(1, PrioritizerFilter.CPU) # <---- next_n(1) # confirm the number requested is honored and the expected node is returned assert len(all_tracking_info) == 1 - assert all_tracking_info[0][1] == cpu_hosts[1] + assert all_tracking_info[0].hostname == cpu_hosts[1] # use next_n w/value that exceeds available number of open nodes - all_tracking_info = p.next_n( - 5, PrioritizerFilter.CPU - ) # <---- 3 direct increments in setup, 1 out of next_n(1), 4 left + # 3 direct increments in setup, 1 out of next_n(1), 4 left + all_tracking_info = p.next_n(5, PrioritizerFilter.CPU) # confirm that no nodes are returned, even though 4 out of 5 requested are available assert len(all_tracking_info) == 0 @@ -325,35 +331,38 @@ def test_node_prioritizer_indirect_direct_interleaved_increments() -> None: p.increment(cpu_hosts[7]) tracking_info = p.get_tracking_info(gpu_hosts[1]) - assert tracking_info[0] == 1 + assert tracking_info.num_refs == 1 tracking_info = p.get_tracking_info(gpu_hosts[3]) - assert tracking_info[0] == 2 + assert tracking_info.num_refs == 2 + + nodes = [n for n in p._nodes.values() if n.num_refs == 0 and n.num_gpus == 0] # we should skip the 0-th item in the heap due to direct increment tracking_info = p.next(PrioritizerFilter.CPU) - assert tracking_info[0] == 1 - assert tracking_info[1] == cpu_hosts[1] + assert tracking_info.num_refs == 1 + # confirm we get a cpu node + assert "cpu-node" in tracking_info.hostname # this should pull the next item right out tracking_info = p.next(PrioritizerFilter.CPU) - assert tracking_info[0] == 1 - assert tracking_info[1] == cpu_hosts[2] - - # we should step over the 3-th node on this iteration - tracking_info = p.next(PrioritizerFilter.CPU) - assert tracking_info[0] == 1 - assert tracking_info[1] == cpu_hosts[4] + assert tracking_info.num_refs == 1 + assert "cpu-node" in tracking_info.hostname # ensure we pull from gpu nodes and the 0th item is returned tracking_info = p.next(PrioritizerFilter.GPU) - assert tracking_info[0] == 1 - assert tracking_info[1] == gpu_hosts[0] + assert tracking_info.num_refs == 1 + assert "gpu-node" in tracking_info.hostname + + # we should step over the 3-th node on this iteration + tracking_info = p.next(PrioritizerFilter.CPU) + assert tracking_info.num_refs == 1 + assert "cpu-node" in tracking_info.hostname # and ensure that heap also steps over a direct increment tracking_info = p.next(PrioritizerFilter.GPU) - assert tracking_info[0] == 1 - assert tracking_info[1] == gpu_hosts[2] + assert tracking_info.num_refs == 1 + assert "gpu-node" in tracking_info.hostname # and another GPU request should return nothing tracking_info = p.next(PrioritizerFilter.GPU) @@ -381,7 +390,7 @@ def test_node_prioritizer_decrement_floor() -> None: for node in nodes: tracking_info = p.get_tracking_info(node.hostname) - assert tracking_info[0] == 0 + assert tracking_info.num_refs == 0 @pytest.mark.parametrize("num_requested", [1, 2, 3]) @@ -430,10 +439,13 @@ def test_node_prioritizer_multi_increment_subheap_assigned() -> None: p.increment(cpu_hosts[0]) p.increment(cpu_hosts[2]) - hostnames = [cpu_hosts[0], cpu_hosts[2]] + hostnames = [ + cpu_hosts[0], + "x" + cpu_hosts[2], + ] # <--- we can't get 2 from 1 valid node name # request n == {num_requested} nodes from set of 3 available - num_requested = 1 + num_requested = 2 all_tracking_info = p.next_n_from(num_requested, hostnames) # w/0,2 assigned, nothing can be returned From 1a74f809c9f85639f1bcac4cc73d0747ea1b0297 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Fri, 9 Aug 2024 18:50:10 -0500 Subject: [PATCH 09/24] typing fixes --- .../_core/launcher/dragon/dragonBackend.py | 4 +- smartsim/_core/launcher/dragon/pqueue.py | 91 ++++++++++--------- 2 files changed, 52 insertions(+), 43 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index 0e8a5fdaa..ed7794aab 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -213,12 +213,12 @@ def allocated_hosts(self) -> dict[str, str]: return self._allocated_hosts @property - def free_hosts(self) -> t.List[str]: + def free_hosts(self) -> t.Sequence[str]: """Find hosts that do not have a step assigned :returns: List of host names""" with self._queue_lock: - return [node.hostname for node in self._prioritizer.unassigned()] + return list(map(lambda x: x.hostname, self._prioritizer.unassigned())) @property def group_infos(self) -> dict[str, ProcessGroupInfo]: diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index ba036d92d..9e480cc41 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -38,7 +38,7 @@ @dataclass class _TrackedNode: - """Minimum Node API required to support the NodePrioritizer""" + """Node API required to have support in the NodePrioritizer""" num_cpus: int """The number of CPUs available on this node""" @@ -51,49 +51,45 @@ class _TrackedNode: tracking: t.Set[str] = field(default_factory=set) """The unique identifiers of processes using this node""" - allocated_cpus: t.Set[int] = field(default_factory=set) - """The CPU indices allocated on this node""" - allocated_gpus: t.Set[int] = field(default_factory=set) - """The GPU indices allocated on this node""" dirty: bool = False """Flag indicating that the node has been updated""" def add( self, tracking_id: t.Optional[str] = None, - cpus: t.Optional[t.Sequence[int]] = None, - gpus: t.Optional[t.Sequence[int]] = None, ) -> None: + """Modify the node as needed to track the removal of a process + + :tracking_id: (optional) a unique task identifier executing on the node + to add""" if tracking_id in self.tracking: raise ValueError("Attempted adding task more than once") self.num_refs = self.num_refs + 1 if tracking_id: self.tracking = self.tracking.union({tracking_id}) - if cpus: - self.allocated_cpus.update(set(cpus)) - if gpus: - self.allocated_gpus.update(set(gpus)) self.dirty = True def remove( self, tracking_id: t.Optional[str] = None, - cpus: t.Optional[t.Sequence[int]] = None, - gpus: t.Optional[t.Sequence[int]] = None, ) -> None: + """Modify the node as needed to track the removal of a process + + :tracking_id: (optional) a unique task identifier executing on the node + to remove""" if tracking_id and tracking_id not in self.tracking: raise ValueError("Attempted removal of untracked item") self.num_refs = max(self.num_refs - 1, 0) self.tracking = self.tracking - {tracking_id} - if cpus: - self.allocated_cpus.difference_update(set(cpus)) - if gpus: - self.allocated_gpus.difference_update(set(gpus)) self.dirty = True def __lt__(self, other: "_TrackedNode") -> bool: + """Comparison operator used to evaluate the ordering of nodes within + the prioritizer. This comparison only considers reference counts. + + :param other: Another node to compare against""" if self.num_refs < other.num_refs: return True @@ -108,18 +104,23 @@ class PrioritizerFilter(str, enum.Enum): class Node(t.Protocol): - """Minimum Node API required to support the NodePrioritizer""" + """Base Node API required to support the NodePrioritizer""" hostname: str + """The hostname of the node""" num_cpus: int + """The number of CPUs in the node""" num_gpus: int + """The number of GPUs in the node""" class NodeReferenceCount(t.Protocol): """Contains details pertaining to references to a node""" hostname: str + """The hostname of the node""" num_refs: int + """The number of jobs assigned to the node""" class NodePrioritizer: @@ -134,13 +135,10 @@ def __init__(self, nodes: t.List[Node], lock: threading.RLock) -> None: self._lock = lock """Lock used to ensure thread safe changes of the reference counters""" - self._cpu_refs: t.List[_TrackedNode] = [] """Track reference counts to CPU-only nodes""" - self._gpu_refs: t.List[_TrackedNode] = [] """Track reference counts to GPU nodes""" - self._nodes: t.Dict[str, _TrackedNode] = {} self._initialize_reference_counters(nodes) @@ -162,21 +160,15 @@ def _initialize_reference_counters(self, nodes: t.List[Node]) -> None: else: self._cpu_refs.append(tracked) - # def _update_ref_count(self, host: str, updated_ref_count: _TrackedNode) -> None: - # """Updates the shared _NodeRefCount instance to keep each - # reference (cpu ref, gpu ref, all refs) in sync""" - # node = self._nodes[host] - - # node.num_refs = updated_ref_count[0] - # node.dirty = updated_ref_count[2] - def increment( self, host: str, tracking_id: t.Optional[str] = None ) -> NodeReferenceCount: """Directly increment the reference count of a given node and ensure the ref counter is marked as dirty to trigger a reordering on retrieval - :param host: a hostname that should have a reference counter selected""" + :param host: a hostname that should have a reference counter selected + :tracking_id: (optional) a unique task identifier executing on the node + to add""" with self._lock: tracked_node = self._nodes[host] tracked_node.add(tracking_id) @@ -196,7 +188,11 @@ def get_tracking_info(self, host: str) -> NodeReferenceCount: """Returns the reference counter information for a single node :param host: a hostname that should have a reference counter selected - :returns: a reference counter""" + :returns: a reference counter for the node + :raises ValueError: if the hostname is not in the set of managed nodes""" + if host not in self._nodes: + raise ValueError("The supplied hostname was not found") + return self._nodes[host] def decrement( @@ -205,7 +201,9 @@ def decrement( """Directly increment the reference count of a given node and ensure the ref counter is marked as dirty to trigger a reordering - :param host: a hostname that should have a reference counter decremented""" + :param host: a hostname that should have a reference counter decremented + :tracking_id: (optional) a unique task identifier executing on the node + to remove""" with self._lock: tracked_node = self._nodes[host] tracked_node.remove(tracking_id) @@ -243,7 +241,7 @@ def next_from(self, hosts: t.List[str]) -> t.Optional[Node]: sub_heap = self._create_sub_heap(hosts) return self._get_next_available_node(sub_heap) - def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.List[Node]: + def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.Sequence[Node]: """Return the next N available nodes given a set of desired hosts :param num_items: the desird number of nodes to allocate @@ -259,7 +257,9 @@ def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.List[Node]: sub_heap = self._create_sub_heap(hosts) return self._get_next_n_available_nodes(num_items, sub_heap) - def unassigned(self, heap: t.Optional[t.List[_TrackedNode]] = None) -> t.List[Node]: + def unassigned( + self, heap: t.Optional[t.List[_TrackedNode]] = None + ) -> t.Sequence[Node]: """Select nodes that are currently not assigned a task :param heap: (optional) a subset of the node heap to consider @@ -267,9 +267,15 @@ def unassigned(self, heap: t.Optional[t.List[_TrackedNode]] = None) -> t.List[No if heap is None: heap = list(self._nodes.values()) - return list(filter(lambda x: x.num_refs == 0, heap)) + nodes: t.List[Node] = [] + for item in heap: + if item.num_refs == 0: + nodes.append(item) + return nodes - def assigned(self, heap: t.Optional[t.List[_TrackedNode]] = None) -> t.List[Node]: + def assigned( + self, heap: t.Optional[t.List[_TrackedNode]] = None + ) -> t.Sequence[Node]: """Helper method to identify the nodes that are currently assigned :param heap: (optional) a subset of the node heap to consider @@ -277,7 +283,12 @@ def assigned(self, heap: t.Optional[t.List[_TrackedNode]] = None) -> t.List[Node if heap is None: heap = list(self._nodes.values()) - return list(filter(lambda x: x.num_refs == 1, heap)) + nodes: t.List[Node] = [] + for item in heap: + if item.num_refs == 1: + nodes.append(item) + return nodes + # return list(filter(lambda x: x.num_refs == 1, heap)) def _check_satisfiable_n( self, num_items: int, heap: t.Optional[t.List[_TrackedNode]] = None @@ -342,9 +353,6 @@ def _get_next_available_node(self, heap: t.List[_TrackedNode]) -> t.Optional[Nod if original_ref_count > 0: return None - if not tracking_info: - return None - return tracking_info def _get_next_n_available_nodes( @@ -355,7 +363,8 @@ def _get_next_n_available_nodes( """Find the next N available nodes w/least amount of references using the supplied filter to target a specific node capability - :param n: number of nodes to reserve + :param num_items: number of nodes to reserve + :param heap: (optional) a subset of the node heap to consider :returns: a list of reference counters for a available nodes if enough unassigned nodes exists, `None` otherwise :raises ValueError: if the number of requetsed nodes is not a positive integer From 27a01e7fc72096426e73829e128dac38a3d5b83c Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 12 Aug 2024 12:19:24 -0500 Subject: [PATCH 10/24] avoid conditional heap sorting and dirty iteration --- smartsim/_core/launcher/dragon/pqueue.py | 185 +++++++++++++++-------- tests/test_node_prioritizer.py | 21 ++- 2 files changed, 132 insertions(+), 74 deletions(-) diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index 9e480cc41..a1b092c89 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -36,23 +36,69 @@ logger = get_logger(__name__) -@dataclass -class _TrackedNode: - """Node API required to have support in the NodePrioritizer""" +@dataclass(frozen=True) +class _Node: + """Contains information on the capabilities supported by a node""" + hostname: str + """The hostname of this node""" num_cpus: int """The number of CPUs available on this node""" num_gpus: int """The number of GPUs available on this node""" - hostname: str - """The hostname of this node""" - num_refs: int = 0 - """The number of processes currently using this node""" - tracking: t.Set[str] = field(default_factory=set) - """The unique identifiers of processes using this node""" - dirty: bool = False - """Flag indicating that the node has been updated""" + +class _TrackedNode: + """Node API required to have support in the NodePrioritizer""" + + def __init__(self, node: _Node) -> None: + self._node = node + """The node being tracked""" + self._num_refs = 0 + """The number of references to the tracked node""" + self._tracking: t.Set[str] = set() + """The unique identifiers of processes using this node""" + self._is_dirty = False + """Flag indicating that tracking information has been modified""" + + @property + def hostname(self) -> str: + """The hostname of the node being reference counted""" + return self._node.hostname + + @property + def num_cpus(self) -> int: + """The number of CPUs of the node being reference counted""" + return self._node.num_cpus + + @property + def num_gpus(self) -> int: + """The number of GPUs of the node being reference counted""" + return self._node.num_gpus + + @property + def num_refs(self) -> int: + """The number of processes currently using the node""" + return self._num_refs + + @property + def is_assigned(self) -> int: + """Returns True if no references are currently being counted""" + return self._num_refs > 0 + + @property + def tracking(self) -> t.Set[str]: + """The set of currently running processes of the node""" + return self._tracking + + @property + def is_dirty(self) -> bool: + """The current modification status of the tracking information""" + return self._is_dirty + + def clean(self) -> None: + """Mark the node as unmodified""" + self._is_dirty = False def add( self, @@ -60,15 +106,15 @@ def add( ) -> None: """Modify the node as needed to track the removal of a process - :tracking_id: (optional) a unique task identifier executing on the node + :param tracking_id: (optional) a unique task identifier executing on the node to add""" if tracking_id in self.tracking: raise ValueError("Attempted adding task more than once") - self.num_refs = self.num_refs + 1 + self._num_refs = self._num_refs + 1 if tracking_id: self.tracking = self.tracking.union({tracking_id}) - self.dirty = True + self._is_dirty = True def remove( self, @@ -76,14 +122,15 @@ def remove( ) -> None: """Modify the node as needed to track the removal of a process - :tracking_id: (optional) a unique task identifier executing on the node + :param tracking_id: (optional) a unique task identifier executing on the node to remove""" if tracking_id and tracking_id not in self.tracking: raise ValueError("Attempted removal of untracked item") - self.num_refs = max(self.num_refs - 1, 0) - self.tracking = self.tracking - {tracking_id} - self.dirty = True + self._num_refs = max(self._num_refs - 1, 0) + if tracking_id: + self._tracking = self._tracking - {tracking_id} + self._is_dirty = True def __lt__(self, other: "_TrackedNode") -> bool: """Comparison operator used to evaluate the ordering of nodes within @@ -148,10 +195,8 @@ def _initialize_reference_counters(self, nodes: t.List[Node]) -> None: :param nodes: node attribute information for initializing the priorizer""" for node in nodes: - # initialize all node counts to 0 and mark the entries "is_dirty=False" - tracked = _TrackedNode( - node.num_cpus, node.num_gpus, node.hostname, 0, set() - ) + # create a set of reference counters for the nodes + tracked = _TrackedNode(node) self._nodes[node.hostname] = tracked # for O(1) access @@ -167,7 +212,7 @@ def increment( ref counter is marked as dirty to trigger a reordering on retrieval :param host: a hostname that should have a reference counter selected - :tracking_id: (optional) a unique task identifier executing on the node + :param tracking_id: (optional) a unique task identifier executing on the node to add""" with self._lock: tracked_node = self._nodes[host] @@ -202,8 +247,7 @@ def decrement( ref counter is marked as dirty to trigger a reordering :param host: a hostname that should have a reference counter decremented - :tracking_id: (optional) a unique task identifier executing on the node - to remove""" + :param tracking_id: (optional) unique task identifier to remove""" with self._lock: tracked_node = self._nodes[host] tracked_node.remove(tracking_id) @@ -244,7 +288,7 @@ def next_from(self, hosts: t.List[str]) -> t.Optional[Node]: def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.Sequence[Node]: """Return the next N available nodes given a set of desired hosts - :param num_items: the desird number of nodes to allocate + :param num_items: the desired number of nodes to allocate :param hosts: a list of hostnames used to filter the available nodes :returns: a list of reference counts :raises ValueError: if no host names are provided""" @@ -319,38 +363,44 @@ def _check_satisfiable_n( return True - def _get_next_available_node(self, heap: t.List[_TrackedNode]) -> t.Optional[Node]: + def _get_next_available_node( + self, + heap: t.List[_TrackedNode], + tracking_id: t.Optional[str] = None, + ) -> t.Optional[Node]: """Finds the next node w/the least amount of running processes and ensures that any elements that were directly updated are updated in the priority structure before being made available :param heap: (optional) a subset of the node heap to consider + :param tracking_id: (optional) unique task identifier to remove :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" tracking_info: t.Optional[_TrackedNode] = None with self._lock: - tracking_info = heapq.heappop(heap) - is_dirty = tracking_info.dirty - - while is_dirty: - if is_dirty: - # mark dirty items clean and place back into heap to be sorted - tracking_info.dirty = False - heapq.heappush(heap, tracking_info) + # re-sort the heap to handle any tracking changes + if any(node.is_dirty for node in heap): + heapq.heapify(heap) - tracking_info = heapq.heappop(heap) - is_dirty = tracking_info.dirty + # grab the min node from the heap + tracking_info = heapq.heappop(heap) - original_ref_count = tracking_info.num_refs - if original_ref_count == 0: - # increment the ref count before putting back onto heap - tracking_info.num_refs += 1 + # the node is available if it has no assigned tasks + is_assigned = tracking_info.is_assigned + if not is_assigned: + # track the new process on the node + tracking_info.add(tracking_id) + # add the node that was popped back into the heap heapq.heappush(heap, tracking_info) - # next available must enforce only "open" return nodes - if original_ref_count > 0: + # mark all nodes as clean now that everything is updated & sorted + for node in heap: + node.clean() + + # next available must only return previously unassigned nodes + if is_assigned: return None return tracking_info @@ -359,6 +409,7 @@ def _get_next_n_available_nodes( self, num_items: int, heap: t.List[_TrackedNode], + tracking_id: t.Optional[str] = None, ) -> t.List[Node]: """Find the next N available nodes w/least amount of references using the supplied filter to target a specific node capability @@ -367,7 +418,7 @@ def _get_next_n_available_nodes( :param heap: (optional) a subset of the node heap to consider :returns: a list of reference counters for a available nodes if enough unassigned nodes exists, `None` otherwise - :raises ValueError: if the number of requetsed nodes is not a positive integer + :raises ValueError: if the number of requested nodes is not a positive integer """ next_nodes: t.List[Node] = [] @@ -378,7 +429,7 @@ def _get_next_n_available_nodes( return next_nodes while len(next_nodes) < num_items: - next_node = self._get_next_available_node(heap) + next_node = self._get_next_available_node(heap, tracking_id) if next_node: next_nodes.append(next_node) else: @@ -386,6 +437,21 @@ def _get_next_n_available_nodes( return next_nodes + def _get_filtered_heap( + self, filter_on: t.Optional[PrioritizerFilter] = None + ) -> t.List[_TrackedNode]: + """Helper method to select the set of nodes to include in a filtered + heap. + + :param filter_on: A list of nodes that satisfy the filter. If no + filter is supplied, all nodes are returned""" + if filter_on == PrioritizerFilter.GPU: + return self._gpu_refs + elif filter_on == PrioritizerFilter.CPU: + return self._cpu_refs + + return self._all_refs() + def next(self, filter_on: t.Optional[PrioritizerFilter] = None) -> t.Optional[Node]: """Find the next node available w/least amount of references using the supplied filter to target a specific node capability @@ -393,20 +459,14 @@ def next(self, filter_on: t.Optional[PrioritizerFilter] = None) -> t.Optional[No :param filter_on: the subset of nodes to query for available nodes :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" - if filter_on == PrioritizerFilter.GPU: - heap = self._gpu_refs - elif filter_on == PrioritizerFilter.CPU: - heap = self._cpu_refs - else: - heap = self._all_refs() - - if node := self._get_next_available_node(heap): - return node - - return None + heap = self._get_filtered_heap(filter_on) + return self._get_next_available_node(heap) def next_n( - self, num_items: int = 1, filter_on: t.Optional[PrioritizerFilter] = None + self, + num_items: int = 1, + filter_on: t.Optional[PrioritizerFilter] = None, + tracking_id: t.Optional[str] = None, ) -> t.List[Node]: """Find the next N available nodes w/least amount of references using the supplied filter to target a specific node capability @@ -414,12 +474,5 @@ def next_n( :param num_items: number of nodes to reserve :param filter_on: the subset of nodes to query for available nodes :returns: Collection of reserved nodes""" - heap = self._cpu_refs - if filter_on == PrioritizerFilter.GPU: - heap = self._gpu_refs - elif filter_on == PrioritizerFilter.CPU: - heap = self._cpu_refs - else: - heap = self._all_refs() - - return self._get_next_n_available_nodes(num_items, heap) + heap = self._get_filtered_heap(filter_on) + return self._get_next_n_available_nodes(num_items, heap, tracking_id) diff --git a/tests/test_node_prioritizer.py b/tests/test_node_prioritizer.py index 2f4a15ea2..eebd989f4 100644 --- a/tests/test_node_prioritizer.py +++ b/tests/test_node_prioritizer.py @@ -41,10 +41,10 @@ class MockNode: - def __init__(self, num_cpus: int, num_gpus: int, hostname: str) -> None: + def __init__(self, hostname: str, num_cpus: int, num_gpus: int) -> None: + self.hostname = hostname self.num_cpus = num_cpus self.num_gpus = num_gpus - self.hostname = hostname def mock_node_hosts( @@ -60,8 +60,8 @@ def mock_node_builder(num_cpu_nodes: int, num_gpu_nodes: int) -> t.List[MockNode nodes = [] cpu_hosts, gpu_hosts = mock_node_hosts(num_cpu_nodes, num_gpu_nodes) - nodes.extend(MockNode(4, 0, hostname) for hostname in cpu_hosts) - nodes.extend(MockNode(4, 4, hostname) for hostname in gpu_hosts) + nodes.extend(MockNode(hostname, 4, 0) for hostname in cpu_hosts) + nodes.extend(MockNode(hostname, 4, 4) for hostname in gpu_hosts) return nodes @@ -114,7 +114,7 @@ def test_node_prioritizer_init_ok(num_cpu_nodes: int, num_gpu_nodes: int) -> Non assert tracking_info.num_refs == 0 # show that the node is created and marked as not dirty (unchanged) - # assert tracking_info.dirty == False + # assert tracking_info.is_dirty == False # iterate through known cpu node keys and verify prioritizer initialization for hostname in cpu_hosts: @@ -215,7 +215,7 @@ def test_node_prioritizer_indirect_increment() -> None: assert tracking_info.num_refs > 0 # <--- ref count should now be > 0 # we expect it to give back only "clean" nodes from next* - assert tracking_info.dirty == False # NOTE: this is "hidden" by protocol + assert tracking_info.is_dirty == False # NOTE: this is "hidden" by protocol # every node should be incremented now. prioritizer shouldn't have anything to give tracking_info = p.next(PrioritizerFilter.CPU) @@ -271,9 +271,14 @@ def test_node_prioritizer_multi_increment() -> None: # use next_n w/the minimum allowed value all_tracking_info = p.next_n(1, PrioritizerFilter.CPU) # <---- next_n(1) - # confirm the number requested is honored and the expected node is returned + # confirm the number requested is honored assert len(all_tracking_info) == 1 - assert all_tracking_info[0].hostname == cpu_hosts[1] + # ensure no unavailable node is returned + assert all_tracking_info[0].hostname not in [ + cpu_hosts[0], + cpu_hosts[2], + cpu_hosts[4], + ] # use next_n w/value that exceeds available number of open nodes # 3 direct increments in setup, 1 out of next_n(1), 4 left From e370bfdc25c0a3b883e88b0eede91d52ba110536 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 12 Aug 2024 12:45:54 -0500 Subject: [PATCH 11/24] type fixes --- smartsim/_core/launcher/dragon/pqueue.py | 79 +++++++++++------------- 1 file changed, 37 insertions(+), 42 deletions(-) diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index a1b092c89..2a55008ae 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -28,7 +28,6 @@ import heapq import threading import typing as t -from dataclasses import dataclass, field from smartsim.error.errors import SmartSimError from smartsim.log import get_logger @@ -36,27 +35,43 @@ logger = get_logger(__name__) -@dataclass(frozen=True) -class _Node: - """Contains information on the capabilities supported by a node""" +class Node(t.Protocol): + """Base Node API required to support the NodePrioritizer""" + + @property + def hostname(self) -> str: + """The hostname of the node""" + + @property + def num_cpus(self) -> int: + """The number of CPUs in the node""" - hostname: str - """The hostname of this node""" - num_cpus: int - """The number of CPUs available on this node""" - num_gpus: int - """The number of GPUs available on this node""" + @property + def num_gpus(self) -> int: + """The number of GPUs in the node""" + + +class NodeReferenceCount(t.Protocol): + """Contains details pertaining to references to a node""" + + @property + def hostname(self) -> str: + """The hostname of the node""" + + @property + def num_refs(self) -> int: + """The number of jobs assigned to the node""" class _TrackedNode: """Node API required to have support in the NodePrioritizer""" - def __init__(self, node: _Node) -> None: + def __init__(self, node: Node) -> None: self._node = node """The node being tracked""" self._num_refs = 0 """The number of references to the tracked node""" - self._tracking: t.Set[str] = set() + self._assigned_tasks: t.Set[str] = set() """The unique identifiers of processes using this node""" self._is_dirty = False """Flag indicating that tracking information has been modified""" @@ -87,9 +102,9 @@ def is_assigned(self) -> int: return self._num_refs > 0 @property - def tracking(self) -> t.Set[str]: + def assigned_tasks(self) -> t.Set[str]: """The set of currently running processes of the node""" - return self._tracking + return self._assigned_tasks @property def is_dirty(self) -> bool: @@ -104,16 +119,16 @@ def add( self, tracking_id: t.Optional[str] = None, ) -> None: - """Modify the node as needed to track the removal of a process + """Modify the node as needed to track the addition of a process :param tracking_id: (optional) a unique task identifier executing on the node to add""" - if tracking_id in self.tracking: + if tracking_id in self.assigned_tasks: raise ValueError("Attempted adding task more than once") self._num_refs = self._num_refs + 1 if tracking_id: - self.tracking = self.tracking.union({tracking_id}) + self._assigned_tasks = self._assigned_tasks.union({tracking_id}) self._is_dirty = True def remove( @@ -124,12 +139,12 @@ def remove( :param tracking_id: (optional) a unique task identifier executing on the node to remove""" - if tracking_id and tracking_id not in self.tracking: + if tracking_id and tracking_id not in self.assigned_tasks: raise ValueError("Attempted removal of untracked item") self._num_refs = max(self._num_refs - 1, 0) if tracking_id: - self._tracking = self._tracking - {tracking_id} + self._assigned_tasks = self._assigned_tasks - {tracking_id} self._is_dirty = True def __lt__(self, other: "_TrackedNode") -> bool: @@ -150,26 +165,6 @@ class PrioritizerFilter(str, enum.Enum): GPU = enum.auto() -class Node(t.Protocol): - """Base Node API required to support the NodePrioritizer""" - - hostname: str - """The hostname of the node""" - num_cpus: int - """The number of CPUs in the node""" - num_gpus: int - """The number of GPUs in the node""" - - -class NodeReferenceCount(t.Protocol): - """Contains details pertaining to references to a node""" - - hostname: str - """The hostname of the node""" - num_refs: int - """The number of jobs assigned to the node""" - - class NodePrioritizer: def __init__(self, nodes: t.List[Node], lock: threading.RLock) -> None: """Initialize the prioritizer @@ -311,7 +306,7 @@ def unassigned( if heap is None: heap = list(self._nodes.values()) - nodes: t.List[Node] = [] + nodes: t.List[_TrackedNode] = [] for item in heap: if item.num_refs == 0: nodes.append(item) @@ -327,7 +322,7 @@ def assigned( if heap is None: heap = list(self._nodes.values()) - nodes: t.List[Node] = [] + nodes: t.List[_TrackedNode] = [] for item in heap: if item.num_refs == 1: nodes.append(item) @@ -447,7 +442,7 @@ def _get_filtered_heap( filter is supplied, all nodes are returned""" if filter_on == PrioritizerFilter.GPU: return self._gpu_refs - elif filter_on == PrioritizerFilter.CPU: + if filter_on == PrioritizerFilter.CPU: return self._cpu_refs return self._all_refs() From a1b0eed1da2efbe22eef0b619d2c3820b31fed54 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 12 Aug 2024 13:29:18 -0500 Subject: [PATCH 12/24] support step colocation in dragon backend --- .../_core/launcher/dragon/dragonBackend.py | 23 +++++++------ smartsim/_core/launcher/dragon/pqueue.py | 32 +++++++++++++++---- 2 files changed, 39 insertions(+), 16 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index ed7794aab..5efcdd7f1 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -205,7 +205,7 @@ def hosts(self) -> list[str]: # todo: remove @property - def allocated_hosts(self) -> dict[str, str]: + def allocated_hosts(self) -> dict[str, t.Set[str]]: """A map of host names to the step id executing on a host :returns: Dictionary with host name as key and step id as value""" @@ -240,9 +240,7 @@ def _initialize_hosts(self) -> None: """List of cpu-count by node""" self._gpus = [node.num_gpus for node in self._nodes] """List of gpu-count by node""" - # NOTE: self._allocated_hosts does NOT support tracking colocated items - # and it must be converted into a list of step IDs in the future. Now? - self._allocated_hosts: t.Dict[str, str] = {} # todo: rename to _host_tasks? + self._allocated_hosts: t.Dict[str, t.Set[str]] = collections.defaultdict(set) """Mapping of hosts to an assigned, running step ID""" self._ref_map: t.Dict[str, _NodeRefCount] = {} """Map node names to a ref counter for direct access""" @@ -456,12 +454,12 @@ def _allocate_step( hosts = [host for host in request.hostlist.split(",") if host] if hosts: - nodes = self._prioritizer.next_n_from(num_hosts, hosts) + nodes = self._prioritizer.next_n_from(num_hosts, hosts, step_id) else: filter_on: t.Optional[PrioritizerFilter] = None if request.policy and request.policy.gpu_affinity: filter_on = PrioritizerFilter.GPU - nodes = self._prioritizer.next_n(num_hosts, filter_on) + nodes = self._prioritizer.next_n(num_hosts, filter_on, step_id) if len(nodes) < num_hosts: # exit if the prioritizer can't identify enough nodes @@ -469,8 +467,9 @@ def _allocate_step( to_allocate = [node.hostname for node in nodes] # track assigning this step to each node + for hostname in to_allocate: - self._allocated_hosts[hostname] = step_id + self._allocated_hosts[hostname].add(step_id) return to_allocate @@ -750,6 +749,10 @@ def _refresh_statuses(self) -> None: if terminated: logger.debug(f"{terminated=}") + # remove all the terminated steps from all hosts + for host in list(self._allocated_hosts.keys()): + self._allocated_hosts[host].difference_update(set(terminated)) + for step_id in terminated: self._running_steps.remove(step_id) self._completed_steps.append(step_id) @@ -758,10 +761,12 @@ def _refresh_statuses(self) -> None: for host in group_info.hosts: logger.debug(f"Releasing host {host}") try: - self._allocated_hosts.pop(host) # todo: remove? + # stop tracking any host no longer running steps + if not self._allocated_hosts[host]: + self._allocated_hosts.pop(host) except KeyError: logger.error(f"Tried to free a non-allocated host: {host}") - self._prioritizer.decrement(host) + self._prioritizer.decrement(host, step_id) group_info.process_group = None group_info.redir_workers = None diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index 2a55008ae..ddd106037 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -268,23 +268,34 @@ def _create_sub_heap(self, hosts: t.List[str]) -> t.List[_TrackedNode]: return nodes_tracking_info - def next_from(self, hosts: t.List[str]) -> t.Optional[Node]: + def next_from( + self, + hosts: t.List[str], + tracking_id: t.Optional[str] = None, + ) -> t.Optional[Node]: """Return the next node available given a set of desired hosts :param hosts: a list of hostnames used to filter the available nodes + :param tracking_id: (optional) unique task identifier to track :returns: a list of assigned reference counters :raises ValueError: if no host names are provided""" if not hosts or len(hosts) == 0: raise ValueError("No host names provided") sub_heap = self._create_sub_heap(hosts) - return self._get_next_available_node(sub_heap) + return self._get_next_available_node(sub_heap, tracking_id) - def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.Sequence[Node]: + def next_n_from( + self, + num_items: int, + hosts: t.List[str], + tracking_id: t.Optional[str] = None, + ) -> t.Sequence[Node]: """Return the next N available nodes given a set of desired hosts :param num_items: the desired number of nodes to allocate :param hosts: a list of hostnames used to filter the available nodes + :param tracking_id: (optional) unique task identifier to track :returns: a list of reference counts :raises ValueError: if no host names are provided""" if not hosts or len(hosts) == 0: @@ -294,7 +305,7 @@ def next_n_from(self, num_items: int, hosts: t.List[str]) -> t.Sequence[Node]: raise ValueError(f"Number of items requested {num_items} is invalid") sub_heap = self._create_sub_heap(hosts) - return self._get_next_n_available_nodes(num_items, sub_heap) + return self._get_next_n_available_nodes(num_items, sub_heap, tracking_id) def unassigned( self, heap: t.Optional[t.List[_TrackedNode]] = None @@ -368,7 +379,7 @@ def _get_next_available_node( the priority structure before being made available :param heap: (optional) a subset of the node heap to consider - :param tracking_id: (optional) unique task identifier to remove + :param tracking_id: (optional) unique task identifier to track :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" tracking_info: t.Optional[_TrackedNode] = None @@ -411,6 +422,7 @@ def _get_next_n_available_nodes( :param num_items: number of nodes to reserve :param heap: (optional) a subset of the node heap to consider + :param tracking_id: (optional) unique task identifier to track :returns: a list of reference counters for a available nodes if enough unassigned nodes exists, `None` otherwise :raises ValueError: if the number of requested nodes is not a positive integer @@ -447,15 +459,20 @@ def _get_filtered_heap( return self._all_refs() - def next(self, filter_on: t.Optional[PrioritizerFilter] = None) -> t.Optional[Node]: + def next( + self, + filter_on: t.Optional[PrioritizerFilter] = None, + tracking_id: t.Optional[str] = None, + ) -> t.Optional[Node]: """Find the next node available w/least amount of references using the supplied filter to target a specific node capability :param filter_on: the subset of nodes to query for available nodes + :param tracking_id: (optional) unique task identifier to track :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" heap = self._get_filtered_heap(filter_on) - return self._get_next_available_node(heap) + return self._get_next_available_node(heap, tracking_id) def next_n( self, @@ -468,6 +485,7 @@ def next_n( :param num_items: number of nodes to reserve :param filter_on: the subset of nodes to query for available nodes + :param tracking_id: (optional) unique task identifier to track :returns: Collection of reserved nodes""" heap = self._get_filtered_heap(filter_on) return self._get_next_n_available_nodes(num_items, heap, tracking_id) From 2ceca98758fb8660b8a05ff01eaca9e1f43c42ec Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 12 Aug 2024 14:00:54 -0500 Subject: [PATCH 13/24] formatting, tweak host cleanup on task term --- .../_core/launcher/dragon/dragonBackend.py | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index 5efcdd7f1..24f54a10f 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -240,8 +240,10 @@ def _initialize_hosts(self) -> None: """List of cpu-count by node""" self._gpus = [node.num_gpus for node in self._nodes] """List of gpu-count by node""" - self._allocated_hosts: t.Dict[str, t.Set[str]] = collections.defaultdict(set) - """Mapping of hosts to an assigned, running step ID""" + self._allocated_hosts: t.Dict[str, t.Set[str]] = collections.defaultdict( + set + ) + """Mapping of hosts to a assigned, running step IDs""" self._ref_map: t.Dict[str, _NodeRefCount] = {} """Map node names to a ref counter for direct access""" self._cpu_refs: t.List[_NodeRefCount] = [] @@ -710,7 +712,7 @@ def _refresh_statuses(self) -> None: stored assigned and unassigned task information""" self._heartbeat() with self._queue_lock: - terminated = [] + terminated: t.Set[str] = set() for step_id in self._running_steps: group_info = self._group_infos[step_id] grp = group_info.process_group @@ -744,14 +746,14 @@ def _refresh_statuses(self) -> None: ) if group_info.status in TERMINAL_STATUSES: - terminated.append(step_id) + terminated.add(step_id) if terminated: logger.debug(f"{terminated=}") # remove all the terminated steps from all hosts for host in list(self._allocated_hosts.keys()): - self._allocated_hosts[host].difference_update(set(terminated)) + self._allocated_hosts[host].difference_update(terminated) for step_id in terminated: self._running_steps.remove(step_id) @@ -760,12 +762,12 @@ def _refresh_statuses(self) -> None: if group_info is not None: for host in group_info.hosts: logger.debug(f"Releasing host {host}") - try: - # stop tracking any host no longer running steps + if host not in self._allocated_hosts: + logger.error(f"Tried to free a non-allocated host: {host}") + else: + # remove any hosts that have had all their steps terminated if not self._allocated_hosts[host]: self._allocated_hosts.pop(host) - except KeyError: - logger.error(f"Tried to free a non-allocated host: {host}") self._prioritizer.decrement(host, step_id) group_info.process_group = None group_info.redir_workers = None From 26ebad27c6186caed5ace0c5f7e9491e55a9b9d2 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 12 Aug 2024 17:36:47 -0500 Subject: [PATCH 14/24] review notes --- smartsim/_core/launcher/dragon/pqueue.py | 36 ++++++++++++---------- smartsim/_core/launcher/step/dragonStep.py | 4 +-- 2 files changed, 21 insertions(+), 19 deletions(-) diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index ddd106037..1754a36b7 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -78,48 +78,50 @@ def __init__(self, node: Node) -> None: @property def hostname(self) -> str: - """The hostname of the node being reference counted""" + """Returns the hostname of the node""" return self._node.hostname @property def num_cpus(self) -> int: - """The number of CPUs of the node being reference counted""" + """Returns the number of CPUs in the node""" return self._node.num_cpus @property def num_gpus(self) -> int: - """The number of GPUs of the node being reference counted""" + """Returns the number of GPUs attached to the node""" return self._node.num_gpus @property def num_refs(self) -> int: - """The number of processes currently using the node""" + """Returns the number of processes currently running on the node""" return self._num_refs @property - def is_assigned(self) -> int: - """Returns True if no references are currently being counted""" + def is_assigned(self) -> bool: + """Returns `True` if no references are currently counted, `False` otherwise""" return self._num_refs > 0 @property def assigned_tasks(self) -> t.Set[str]: - """The set of currently running processes of the node""" + """Returns the set of unique IDs for currently running processes""" return self._assigned_tasks @property def is_dirty(self) -> bool: - """The current modification status of the tracking information""" + """Returns a flag indicating if the reference counter has changed. `True` + if references have been added or removed, `False` otherwise.""" return self._is_dirty def clean(self) -> None: - """Mark the node as unmodified""" + """Marks the node as unmodified""" self._is_dirty = False def add( self, tracking_id: t.Optional[str] = None, ) -> None: - """Modify the node as needed to track the addition of a process + """Update the node to indicate the addition of a process that must be + reference counted. :param tracking_id: (optional) a unique task identifier executing on the node to add""" @@ -135,7 +137,7 @@ def remove( self, tracking_id: t.Optional[str] = None, ) -> None: - """Modify the node as needed to track the removal of a process + """Update the reference counter to indicate the removal of a process. :param tracking_id: (optional) a unique task identifier executing on the node to remove""" @@ -151,7 +153,8 @@ def __lt__(self, other: "_TrackedNode") -> bool: """Comparison operator used to evaluate the ordering of nodes within the prioritizer. This comparison only considers reference counts. - :param other: Another node to compare against""" + :param other: Another node to compare against + :returns: True if this node has fewer references than the other node""" if self.num_refs < other.num_refs: return True @@ -171,6 +174,7 @@ def __init__(self, nodes: t.List[Node], lock: threading.RLock) -> None: :param nodes: node attribute information for initializing the priorizer :param lock: a lock used to ensure threadsafe operations + :raises SmartSimError: if the nodes collection is empty """ if not nodes: raise SmartSimError("Missing nodes to prioritize") @@ -216,7 +220,7 @@ def increment( # self._update_ref_count(host, tracked_node) return tracked_node - def _all_refs(self) -> t.List[_TrackedNode]: + def _heapify_all_refs(self) -> t.List[_TrackedNode]: """Combine the CPU and GPU nodes into a single heap :returns: list of all reference counters""" @@ -238,7 +242,7 @@ def get_tracking_info(self, host: str) -> NodeReferenceCount: def decrement( self, host: str, tracking_id: t.Optional[str] = None ) -> NodeReferenceCount: - """Directly increment the reference count of a given node and ensure the + """Directly decrement the reference count of a given node and ensure the ref counter is marked as dirty to trigger a reordering :param host: a hostname that should have a reference counter decremented @@ -335,7 +339,7 @@ def assigned( nodes: t.List[_TrackedNode] = [] for item in heap: - if item.num_refs == 1: + if item.num_refs > 0: nodes.append(item) return nodes # return list(filter(lambda x: x.num_refs == 1, heap)) @@ -457,7 +461,7 @@ def _get_filtered_heap( if filter_on == PrioritizerFilter.CPU: return self._cpu_refs - return self._all_refs() + return self._heapify_all_refs() def next( self, diff --git a/smartsim/_core/launcher/step/dragonStep.py b/smartsim/_core/launcher/step/dragonStep.py index cf0cc3983..e5caeaa84 100644 --- a/smartsim/_core/launcher/step/dragonStep.py +++ b/smartsim/_core/launcher/step/dragonStep.py @@ -167,9 +167,7 @@ def _write_request_file(self) -> str: run_settings = t.cast(DragonRunSettings, step.step_settings) run_args = run_settings.run_args env = run_settings.env_vars - nodes = int( - run_args.get("nodes", None) or 1 - ) # todo: is this the same use as my new host-list? + nodes = int(run_args.get("nodes", 1) or 1) tasks_per_node = int(run_args.get("tasks-per-node", None) or 1) hosts_csv = str(run_args.get("host-list", "")) From 72b9c1768297b1091c4e5625073e036dc052a58e Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 12 Aug 2024 18:25:03 -0500 Subject: [PATCH 15/24] reduce api surface --- smartsim/_core/launcher/dragon/pqueue.py | 63 +++++++----------------- tests/test_node_prioritizer.py | 28 +++++------ 2 files changed, 31 insertions(+), 60 deletions(-) diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index 1754a36b7..e85ba1f9b 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -251,66 +251,27 @@ def decrement( tracked_node = self._nodes[host] tracked_node.remove(tracking_id) - # self._update_ref_count(host, tracked_node.as_refcount()) return tracked_node - def _create_sub_heap(self, hosts: t.List[str]) -> t.List[_TrackedNode]: + def _create_sub_heap(self, hosts: t.List[str], filter_on: t.Optional[PrioritizerFilter] = None) -> t.List[_TrackedNode]: """Create a new heap from the primary heap with user-specified nodes :param hosts: a list of hostnames used to filter the available nodes :returns: a list of assigned reference counters """ nodes_tracking_info: t.List[_TrackedNode] = [] + heap = self._get_filtered_heap(filter_on) # Collect all the tracking info for the requested nodes... - for host in hosts: - if tracking_info := self._nodes.get(host, None): - nodes_tracking_info.append(tracking_info) + for node in heap: + if not hosts or node.hostname in hosts: + nodes_tracking_info.append(node) # ... and use it to create a new heap from a specified subset of nodes heapq.heapify(nodes_tracking_info) return nodes_tracking_info - def next_from( - self, - hosts: t.List[str], - tracking_id: t.Optional[str] = None, - ) -> t.Optional[Node]: - """Return the next node available given a set of desired hosts - - :param hosts: a list of hostnames used to filter the available nodes - :param tracking_id: (optional) unique task identifier to track - :returns: a list of assigned reference counters - :raises ValueError: if no host names are provided""" - if not hosts or len(hosts) == 0: - raise ValueError("No host names provided") - - sub_heap = self._create_sub_heap(hosts) - return self._get_next_available_node(sub_heap, tracking_id) - - def next_n_from( - self, - num_items: int, - hosts: t.List[str], - tracking_id: t.Optional[str] = None, - ) -> t.Sequence[Node]: - """Return the next N available nodes given a set of desired hosts - - :param num_items: the desired number of nodes to allocate - :param hosts: a list of hostnames used to filter the available nodes - :param tracking_id: (optional) unique task identifier to track - :returns: a list of reference counts - :raises ValueError: if no host names are provided""" - if not hosts or len(hosts) == 0: - raise ValueError("No host names provided") - - if num_items < 1: - raise ValueError(f"Number of items requested {num_items} is invalid") - - sub_heap = self._create_sub_heap(hosts) - return self._get_next_n_available_nodes(num_items, sub_heap, tracking_id) - def unassigned( self, heap: t.Optional[t.List[_TrackedNode]] = None ) -> t.Sequence[Node]: @@ -467,15 +428,20 @@ def next( self, filter_on: t.Optional[PrioritizerFilter] = None, tracking_id: t.Optional[str] = None, + hosts: t.Optional[t.List[str]] = None, ) -> t.Optional[Node]: """Find the next node available w/least amount of references using the supplied filter to target a specific node capability :param filter_on: the subset of nodes to query for available nodes :param tracking_id: (optional) unique task identifier to track + :param hosts: a list of hostnames used to filter the available nodes :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" - heap = self._get_filtered_heap(filter_on) + if hosts is not None and not hosts: + raise ValueError("No host names provided") + + heap = self._create_sub_heap(hosts, filter_on) return self._get_next_available_node(heap, tracking_id) def next_n( @@ -483,6 +449,7 @@ def next_n( num_items: int = 1, filter_on: t.Optional[PrioritizerFilter] = None, tracking_id: t.Optional[str] = None, + hosts: t.Optional[t.List[str]] = None, ) -> t.List[Node]: """Find the next N available nodes w/least amount of references using the supplied filter to target a specific node capability @@ -490,6 +457,10 @@ def next_n( :param num_items: number of nodes to reserve :param filter_on: the subset of nodes to query for available nodes :param tracking_id: (optional) unique task identifier to track + :param hosts: a list of hostnames used to filter the available nodes :returns: Collection of reserved nodes""" - heap = self._get_filtered_heap(filter_on) + if hosts is not None and not hosts: + raise ValueError("No host names provided") + + heap = self._create_sub_heap(hosts, filter_on) return self._get_next_n_available_nodes(num_items, heap, tracking_id) diff --git a/tests/test_node_prioritizer.py b/tests/test_node_prioritizer.py index eebd989f4..70df5a840 100644 --- a/tests/test_node_prioritizer.py +++ b/tests/test_node_prioritizer.py @@ -400,7 +400,7 @@ def test_node_prioritizer_decrement_floor() -> None: @pytest.mark.parametrize("num_requested", [1, 2, 3]) def test_node_prioritizer_multi_increment_subheap(num_requested: int) -> None: - """Verify that retrieving multiple nodes via `next_n_from` API correctly + """Verify that retrieving multiple nodes via `next_n` API correctly increments reference counts and returns appropriate results when requesting an in-bounds number of nodes""" @@ -419,9 +419,9 @@ def test_node_prioritizer_multi_increment_subheap(num_requested: int) -> None: hostnames = [cpu_hosts[0], cpu_hosts[1], cpu_hosts[2], cpu_hosts[3], cpu_hosts[5]] # request n == {num_requested} nodes from set of 3 available - all_tracking_info = p.next_n_from( + all_tracking_info = p.next_n( num_requested, - hostnames, + hosts=hostnames, ) # <---- w/0,2,4 assigned, only 1,3,5 from hostnames can work # all parameterizations should result in a matching output size @@ -429,7 +429,7 @@ def test_node_prioritizer_multi_increment_subheap(num_requested: int) -> None: def test_node_prioritizer_multi_increment_subheap_assigned() -> None: - """Verify that retrieving multiple nodes via `next_n_from` API does + """Verify that retrieving multiple nodes via `next_n` API does not return anything when the number requested cannot be satisfied by the given subheap due to prior assignment""" @@ -451,14 +451,14 @@ def test_node_prioritizer_multi_increment_subheap_assigned() -> None: # request n == {num_requested} nodes from set of 3 available num_requested = 2 - all_tracking_info = p.next_n_from(num_requested, hostnames) + all_tracking_info = p.next_n(num_requested, hosts=hostnames) # w/0,2 assigned, nothing can be returned assert len(all_tracking_info) == 0 -def test_node_prioritizer_empty_subheap_next_from() -> None: - """Verify that retrieving multiple nodes via `next_n_from` API does +def test_node_prioritizer_empty_subheap_next_w_hosts() -> None: + """Verify that retrieving multiple nodes via `next_n` API does not allow an empty host list""" num_cpu_nodes, num_gpu_nodes = 8, 0 @@ -477,13 +477,13 @@ def test_node_prioritizer_empty_subheap_next_from() -> None: # request n == {num_requested} nodes from set of 3 available num_requested = 1 with pytest.raises(ValueError) as ex: - p.next_from(hostnames) + p.next(hosts=hostnames) assert "No host names provided" == ex.value.args[0] -def test_node_prioritizer_empty_subheap_next_n_from() -> None: - """Verify that retrieving multiple nodes via `next_n_from` API does +def test_node_prioritizer_empty_subheap_next_n_w_hosts() -> None: + """Verify that retrieving multiple nodes via `next_n` API does not allow an empty host list""" num_cpu_nodes, num_gpu_nodes = 8, 0 @@ -502,7 +502,7 @@ def test_node_prioritizer_empty_subheap_next_n_from() -> None: # request n == {num_requested} nodes from set of 3 available num_requested = 1 with pytest.raises(ValueError) as ex: - p.next_n_from(num_requested, hostnames) + p.next_n(num_requested, hosts=hostnames) assert "No host names provided" == ex.value.args[0] @@ -531,8 +531,8 @@ def test_node_prioritizer_empty_subheap_next_n(num_requested: int) -> None: @pytest.mark.parametrize("num_requested", [-100, -1, 0]) -def test_node_prioritizer_empty_subheap_next_n_from(num_requested: int) -> None: - """Verify that retrieving multiple nodes via `next_n_from` API does +def test_node_prioritizer_empty_subheap_next_n(num_requested: int) -> None: + """Verify that retrieving multiple nodes via `next_n` API does not allow a request with num_items < 1""" num_cpu_nodes, num_gpu_nodes = 8, 0 @@ -550,6 +550,6 @@ def test_node_prioritizer_empty_subheap_next_n_from(num_requested: int) -> None: # request n == {num_requested} nodes from set of 3 available with pytest.raises(ValueError) as ex: - p.next_n_from(num_requested, hostnames) + p.next_n(num_requested, hosts=hostnames) assert "Number of items requested" in ex.value.args[0] From 833844402a54756126788b4dd4acfa5739ccae49 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Tue, 13 Aug 2024 10:54:07 -0500 Subject: [PATCH 16/24] cleanup extraneous branching per review --- .../_core/launcher/dragon/dragonBackend.py | 12 ++--- smartsim/_core/launcher/dragon/pqueue.py | 51 ++++++++++--------- 2 files changed, 31 insertions(+), 32 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index 24f54a10f..df9aee981 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -455,13 +455,11 @@ def _allocate_step( # convert the comma-separated argument into a real list hosts = [host for host in request.hostlist.split(",") if host] - if hosts: - nodes = self._prioritizer.next_n_from(num_hosts, hosts, step_id) - else: - filter_on: t.Optional[PrioritizerFilter] = None - if request.policy and request.policy.gpu_affinity: - filter_on = PrioritizerFilter.GPU - nodes = self._prioritizer.next_n(num_hosts, filter_on, step_id) + filter_on: t.Optional[PrioritizerFilter] = None + if request.policy and request.policy.gpu_affinity: + filter_on = PrioritizerFilter.GPU + + nodes = self._prioritizer.next_n(num_hosts, filter_on, step_id, hosts) if len(nodes) < num_hosts: # exit if the prioritizer can't identify enough nodes diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index e85ba1f9b..cc2fd38e6 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -123,7 +123,7 @@ def add( """Update the node to indicate the addition of a process that must be reference counted. - :param tracking_id: (optional) a unique task identifier executing on the node + :param tracking_id: a unique task identifier executing on the node to add""" if tracking_id in self.assigned_tasks: raise ValueError("Attempted adding task more than once") @@ -139,7 +139,7 @@ def remove( ) -> None: """Update the reference counter to indicate the removal of a process. - :param tracking_id: (optional) a unique task identifier executing on the node + :param tracking_id: a unique task identifier executing on the node to remove""" if tracking_id and tracking_id not in self.assigned_tasks: raise ValueError("Attempted removal of untracked item") @@ -211,7 +211,7 @@ def increment( ref counter is marked as dirty to trigger a reordering on retrieval :param host: a hostname that should have a reference counter selected - :param tracking_id: (optional) a unique task identifier executing on the node + :param tracking_id: a unique task identifier executing on the node to add""" with self._lock: tracked_node = self._nodes[host] @@ -246,14 +246,18 @@ def decrement( ref counter is marked as dirty to trigger a reordering :param host: a hostname that should have a reference counter decremented - :param tracking_id: (optional) unique task identifier to remove""" + :param tracking_id: unique task identifier to remove""" with self._lock: tracked_node = self._nodes[host] tracked_node.remove(tracking_id) return tracked_node - def _create_sub_heap(self, hosts: t.List[str], filter_on: t.Optional[PrioritizerFilter] = None) -> t.List[_TrackedNode]: + def _create_sub_heap( + self, + hosts: t.Optional[t.List[str]] = None, + filter_on: t.Optional[PrioritizerFilter] = None, + ) -> t.List[_TrackedNode]: """Create a new heap from the primary heap with user-specified nodes :param hosts: a list of hostnames used to filter the available nodes @@ -277,7 +281,7 @@ def unassigned( ) -> t.Sequence[Node]: """Select nodes that are currently not assigned a task - :param heap: (optional) a subset of the node heap to consider + :param heap: a subset of the node heap to consider :returns: a list of reference counts for all unassigned nodes""" if heap is None: heap = list(self._nodes.values()) @@ -293,7 +297,7 @@ def assigned( ) -> t.Sequence[Node]: """Helper method to identify the nodes that are currently assigned - :param heap: (optional) a subset of the node heap to consider + :param heap: a subset of the node heap to consider :returns: a list of reference counts for all assigned nodes""" if heap is None: heap = list(self._nodes.values()) @@ -312,7 +316,7 @@ def _check_satisfiable_n( satisfied by the prioritizer given the set of nodes available :param num_items: the desird number of nodes to allocate - :param heap: (optional) a subset of the node heap to consider + :param heap: a subset of the node heap to consider :returns: True if the request can be fulfilled, False otherwise""" num_nodes = len(self._nodes.keys()) @@ -343,8 +347,8 @@ def _get_next_available_node( ensures that any elements that were directly updated are updated in the priority structure before being made available - :param heap: (optional) a subset of the node heap to consider - :param tracking_id: (optional) unique task identifier to track + :param heap: a subset of the node heap to consider + :param tracking_id: unique task identifier to track :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" tracking_info: t.Optional[_TrackedNode] = None @@ -386,8 +390,8 @@ def _get_next_n_available_nodes( the supplied filter to target a specific node capability :param num_items: number of nodes to reserve - :param heap: (optional) a subset of the node heap to consider - :param tracking_id: (optional) unique task identifier to track + :param heap: a subset of the node heap to consider + :param tracking_id: unique task identifier to track :returns: a list of reference counters for a available nodes if enough unassigned nodes exists, `None` otherwise :raises ValueError: if the number of requested nodes is not a positive integer @@ -401,11 +405,10 @@ def _get_next_n_available_nodes( return next_nodes while len(next_nodes) < num_items: - next_node = self._get_next_available_node(heap, tracking_id) - if next_node: + if next_node := self._get_next_available_node(heap, tracking_id): next_nodes.append(next_node) - else: - break + continue + break return next_nodes @@ -430,19 +433,17 @@ def next( tracking_id: t.Optional[str] = None, hosts: t.Optional[t.List[str]] = None, ) -> t.Optional[Node]: - """Find the next node available w/least amount of references using - the supplied filter to target a specific node capability + """Find the next available node using the supplied filter to target + a specific node capability :param filter_on: the subset of nodes to query for available nodes - :param tracking_id: (optional) unique task identifier to track + :param tracking_id: unique task identifier to track :param hosts: a list of hostnames used to filter the available nodes :returns: a reference counter for an available node if an unassigned node exists, `None` otherwise""" - if hosts is not None and not hosts: - raise ValueError("No host names provided") - - heap = self._create_sub_heap(hosts, filter_on) - return self._get_next_available_node(heap, tracking_id) + if results := self.next_n(1, filter_on, tracking_id, hosts): + return results[0] + return None def next_n( self, @@ -456,7 +457,7 @@ def next_n( :param num_items: number of nodes to reserve :param filter_on: the subset of nodes to query for available nodes - :param tracking_id: (optional) unique task identifier to track + :param tracking_id: unique task identifier to track :param hosts: a list of hostnames used to filter the available nodes :returns: Collection of reserved nodes""" if hosts is not None and not hosts: From bee5eac373d699db936abc5ff55336fd6162920d Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Tue, 13 Aug 2024 10:57:43 -0500 Subject: [PATCH 17/24] remove obsoleted code --- smartsim/_core/launcher/dragon/dragonBackend.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index df9aee981..a5be0f585 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -71,10 +71,6 @@ logger = get_logger(__name__) -# tracking structure for [num_refs, node_name, is_dirty] -_NodeRefCount = t.List[t.Union[int, str, bool]] - - class DragonStatus(str, Enum): ERROR = str(dragon_group_state.Error()) RUNNING = str(dragon_group_state.Running()) @@ -244,12 +240,6 @@ def _initialize_hosts(self) -> None: set ) """Mapping of hosts to a assigned, running step IDs""" - self._ref_map: t.Dict[str, _NodeRefCount] = {} - """Map node names to a ref counter for direct access""" - self._cpu_refs: t.List[_NodeRefCount] = [] - """Track reference counts to CPU-only nodes""" - self._gpu_refs: t.List[_NodeRefCount] = [] - """Track reference counts to GPU nodes""" def __str__(self) -> str: return self.status_message From df664678819708f53ad77e31b6830afc5540cba7 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Tue, 13 Aug 2024 11:53:45 -0500 Subject: [PATCH 18/24] cleanup and minor reversions --- smartsim/_core/launcher/dragon/dragonBackend.py | 9 ++------- smartsim/_core/launcher/step/dragonStep.py | 2 +- smartsim/_core/schemas/dragonRequests.py | 2 +- 3 files changed, 4 insertions(+), 9 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index a5be0f585..c940269c6 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -199,7 +199,6 @@ def hosts(self) -> list[str]: with self._queue_lock: return self._hosts - # todo: remove @property def allocated_hosts(self) -> dict[str, t.Set[str]]: """A map of host names to the step id executing on a host @@ -320,9 +319,7 @@ def _can_honor_policy( an optional error message""" # ensure the policy can be honored if request.policy: - logger.debug(f"{request.policy=}") - logger.debug(f"{self._cpus=}") - logger.debug(f"{self._gpus=}") + logger.debug(f"{request.policy=}{self._cpus=}"){self._gpus=}") if request.policy.cpu_affinity: # make sure some node has enough CPUs @@ -392,9 +389,7 @@ def _can_honor_hosts( valid_hosts = all_hosts.intersection(requested_hosts) invalid_hosts = requested_hosts - valid_hosts - logger.debug(f"{num_nodes=}") - logger.debug(f"{valid_hosts=}") - logger.debug(f"{invalid_hosts=}") + logger.debug(f"{num_nodes=}{valid_hosts=}{invalid_hosts=}") if invalid_hosts: logger.warning(f"Some invalid hostnames were requested: {invalid_hosts}") diff --git a/smartsim/_core/launcher/step/dragonStep.py b/smartsim/_core/launcher/step/dragonStep.py index e5caeaa84..21fdc697c 100644 --- a/smartsim/_core/launcher/step/dragonStep.py +++ b/smartsim/_core/launcher/step/dragonStep.py @@ -167,7 +167,7 @@ def _write_request_file(self) -> str: run_settings = t.cast(DragonRunSettings, step.step_settings) run_args = run_settings.run_args env = run_settings.env_vars - nodes = int(run_args.get("nodes", 1) or 1) + nodes = int(run_args.get("nodes", None) or 1) tasks_per_node = int(run_args.get("tasks-per-node", None) or 1) hosts_csv = str(run_args.get("host-list", "")) diff --git a/smartsim/_core/schemas/dragonRequests.py b/smartsim/_core/schemas/dragonRequests.py index 636375faf..487ea915a 100644 --- a/smartsim/_core/schemas/dragonRequests.py +++ b/smartsim/_core/schemas/dragonRequests.py @@ -64,7 +64,7 @@ def from_run_args( if cpu_arg_value := run_args.get("cpu-affinity", None): cpu_args = str(cpu_arg_value) - # list[int] converted to comma-separated str must split into a list[int] + # run args converted to a string must be split back into a list[int] gpu_affinity = [int(x.strip()) for x in gpu_args.split(",") if x] cpu_affinity = [int(x.strip()) for x in cpu_args.split(",") if x] From 8ce01965089256a2a22d26959588fc3cd32735e0 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Tue, 13 Aug 2024 11:53:55 -0500 Subject: [PATCH 19/24] add dragon run settings tests --- smartsim/settings/dragonRunSettings.py | 28 ++---- tests/test_dragon_runsettings.py | 119 +++++++++++++++++++++++++ 2 files changed, 128 insertions(+), 19 deletions(-) diff --git a/smartsim/settings/dragonRunSettings.py b/smartsim/settings/dragonRunSettings.py index dc8403267..a003bd7d8 100644 --- a/smartsim/settings/dragonRunSettings.py +++ b/smartsim/settings/dragonRunSettings.py @@ -104,25 +104,15 @@ def set_hostlist(self, host_list: t.Union[str, t.List[str]]) -> None: if not host_list: raise ValueError("empty hostlist provided") - if isinstance(host_list, list): - self.run_args["host-list"] = ",".join( - host.replace(" ", "") for host in host_list - ) - return - - # TODO: BREAKING ON PURPOSE! - # Add a test for DragonRunSettings for: - # - set_hotslist (None) - # - set_hostlist([]) - # - set_hostlist(['']) - # - set_hostlist(['abc']) - single - # - set_hostlist(['abc', 'def']) - multi - # - set_hostlist('abc') - single (str) - # - set_hostlist('abc,def']) - multi (str), - # - set_hostlist('abc, def']) - multi (str), - # spaces removal 'x, y' -> 'x','y', not 'x', ' y' - - self.run_args["host-list"] = host_list.replace(" ", "") + if isinstance(host_list, str): + host_list = host_list.replace(" ", "").split(",") + + # strip out all whitespace-only values + cleaned_list = [host.strip() for host in host_list if host and host.strip()] + if not len(cleaned_list) == len(host_list): + raise ValueError(f"invalid names found in hostlist: {host_list}") + + self.run_args["host-list"] = ",".join(cleaned_list) def set_cpu_affinity(self, devices: t.List[int]) -> None: """Set the CPU affinity for this job diff --git a/tests/test_dragon_runsettings.py b/tests/test_dragon_runsettings.py index 34e8510e8..8c7600c74 100644 --- a/tests/test_dragon_runsettings.py +++ b/tests/test_dragon_runsettings.py @@ -96,3 +96,122 @@ def test_dragon_runsettings_gpu_affinity(): # ensure the value is not changed when we extend the list rs.run_args["gpu-affinity"] = "7,8,9" assert rs.run_args["gpu-affinity"] != ",".join(str(val) for val in exp_value) + + +def test_dragon_runsettings_hostlist_null(): + """Verify that passing a null hostlist is treated as a failure""" + rs = DragonRunSettings(exe="sleep", exe_args=["1"]) + + # baseline check that no host list exists + stored_list = rs.run_args.get("host-list", None) + assert stored_list is None + + with pytest.raises(ValueError) as ex: + rs.set_hostlist(None) + + assert "empty hostlist" in ex.value.args[0] + + +def test_dragon_runsettings_hostlist_empty(): + """Verify that passing an empty hostlist is treated as a failure""" + rs = DragonRunSettings(exe="sleep", exe_args=["1"]) + + # baseline check that no host list exists + stored_list = rs.run_args.get("host-list", None) + assert stored_list is None + + with pytest.raises(ValueError) as ex: + rs.set_hostlist([]) + + assert "empty hostlist" in ex.value.args[0] + + +@pytest.mark.parametrize("hostlist_csv", [" ", " , , , ", ",", ",,,"]) +def test_dragon_runsettings_hostlist_whitespace_handling(hostlist_csv: str): + """Verify that passing a hostlist with emptystring host names is treated as a failure""" + rs = DragonRunSettings(exe="sleep", exe_args=["1"]) + + # baseline check that no host list exists + stored_list = rs.run_args.get("host-list", None) + assert stored_list is None + + # empty string as hostname in list + with pytest.raises(ValueError) as ex: + rs.set_hostlist(hostlist_csv) + + assert "invalid names" in ex.value.args[0] + + +@pytest.mark.parametrize( + "hostlist_csv", [[" "], [" ", "", " ", " "], ["", " "], ["", "", "", ""]] +) +def test_dragon_runsettings_hostlist_whitespace_handling_list(hostlist_csv: str): + """Verify that passing a hostlist with emptystring host names contained in a list + is treated as a failure""" + rs = DragonRunSettings(exe="sleep", exe_args=["1"]) + + # baseline check that no host list exists + stored_list = rs.run_args.get("host-list", None) + assert stored_list is None + + # empty string as hostname in list + with pytest.raises(ValueError) as ex: + rs.set_hostlist(hostlist_csv) + + assert "invalid names" in ex.value.args[0] + + +def test_dragon_runsettings_hostlist_as_csv(): + """Verify that a hostlist is stored properly when passing in a CSV string""" + rs = DragonRunSettings(exe="sleep", exe_args=["1"]) + + # baseline check that no host list exists + stored_list = rs.run_args.get("host-list", None) + assert stored_list is None + + hostnames = ["host0", "host1", "host2", "host3", "host4"] + + # set the host list with ideal comma separated values + input0 = ",".join(hostnames) + + # set the host list with a string of comma separated values + # including extra whitespace + input1 = ", ".join(hostnames) + + for hosts_input in [input0, input1]: + rs.set_hostlist(hosts_input) + + stored_list = rs.run_args.get("host-list", None) + assert stored_list + + # confirm that all values from the original list are retrieved + split_stored_list = stored_list.split(",") + assert set(hostnames) == set(split_stored_list) + + +def test_dragon_runsettings_hostlist_as_csv(): + """Verify that a hostlist is stored properly when passing in a CSV string""" + rs = DragonRunSettings(exe="sleep", exe_args=["1"]) + + # baseline check that no host list exists + stored_list = rs.run_args.get("host-list", None) + assert stored_list is None + + hostnames = ["host0", "host1", "host2", "host3", "host4"] + + # set the host list with ideal comma separated values + input0 = ",".join(hostnames) + + # set the host list with a string of comma separated values + # including extra whitespace + input1 = ", ".join(hostnames) + + for hosts_input in [input0, input1]: + rs.set_hostlist(hosts_input) + + stored_list = rs.run_args.get("host-list", None) + assert stored_list + + # confirm that all values from the original list are retrieved + split_stored_list = stored_list.split(",") + assert set(hostnames) == set(split_stored_list) From 34d918e8dd1c24488704d6e592503e0a30a17b7f Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Tue, 13 Aug 2024 11:58:05 -0500 Subject: [PATCH 20/24] typo --- smartsim/_core/launcher/dragon/dragonBackend.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index c940269c6..c52224d7e 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -319,7 +319,7 @@ def _can_honor_policy( an optional error message""" # ensure the policy can be honored if request.policy: - logger.debug(f"{request.policy=}{self._cpus=}"){self._gpus=}") + logger.debug(f"{request.policy=}{self._cpus=}{self._gpus=}") if request.policy.cpu_affinity: # make sure some node has enough CPUs From 2d73740b580ddac2a5a0d9845b1115ff575b2e42 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Tue, 13 Aug 2024 12:07:28 -0500 Subject: [PATCH 21/24] formatting --- smartsim/_core/launcher/dragon/dragonBackend.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index c52224d7e..20182f5d6 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -451,9 +451,9 @@ def _allocate_step( return None to_allocate = [node.hostname for node in nodes] - # track assigning this step to each node for hostname in to_allocate: + # track assigning this step to each node self._allocated_hosts[hostname].add(step_id) return to_allocate From 9f066957630bec499da55eb76b62af7cfef2eb35 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Wed, 14 Aug 2024 14:50:18 -0500 Subject: [PATCH 22/24] remove commented code, fix bad docstrings --- smartsim/_core/launcher/dragon/dragonBackend.py | 6 +++--- smartsim/_core/launcher/dragon/pqueue.py | 5 +---- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index 20182f5d6..2e528092f 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -238,7 +238,7 @@ def _initialize_hosts(self) -> None: self._allocated_hosts: t.Dict[str, t.Set[str]] = collections.defaultdict( set ) - """Mapping of hosts to a assigned, running step IDs""" + """Mapping with hostnames as keys and a set of executing step IDs as the value""" def __str__(self) -> str: return self.status_message @@ -332,7 +332,7 @@ def _can_honor_policy( last_available = max(self._gpus or [-1]) requested = max(request.policy.gpu_affinity) if not any(self._gpus) or requested >= last_available: - print( + logger.warning( f"failed check w/{self._gpus=}, {requested=}, {last_available=}" ) return False, "Cannot satisfy request, not enough GPUs available" @@ -377,7 +377,7 @@ def _can_honor_hosts( # fail if requesting more nodes than the total number available if num_nodes > len(all_hosts): message = f"Cannot satisfy request. {num_nodes} requested nodes" - message += f"exceeds {len(all_hosts)} available." + message += f" exceeds {len(all_hosts)} available." return False, message requested_hosts = all_hosts diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index cc2fd38e6..d160174e6 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -216,8 +216,6 @@ def increment( with self._lock: tracked_node = self._nodes[host] tracked_node.add(tracking_id) - - # self._update_ref_count(host, tracked_node) return tracked_node def _heapify_all_refs(self) -> t.List[_TrackedNode]: @@ -307,7 +305,6 @@ def assigned( if item.num_refs > 0: nodes.append(item) return nodes - # return list(filter(lambda x: x.num_refs == 1, heap)) def _check_satisfiable_n( self, num_items: int, heap: t.Optional[t.List[_TrackedNode]] = None @@ -315,7 +312,7 @@ def _check_satisfiable_n( """Validates that a request for some number of nodes `n` can be satisfied by the prioritizer given the set of nodes available - :param num_items: the desird number of nodes to allocate + :param num_items: the desired number of nodes to allocate :param heap: a subset of the node heap to consider :returns: True if the request can be fulfilled, False otherwise""" num_nodes = len(self._nodes.keys()) From eb182b177d52539d8d4524676c4feea581b36520 Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 19 Aug 2024 10:51:18 -0500 Subject: [PATCH 23/24] fixes per review notes, move poorly-placed class var declarations --- .../_core/launcher/dragon/dragonBackend.py | 21 ++++++++++++------- smartsim/_core/launcher/dragon/pqueue.py | 19 ++++++++++------- smartsim/settings/dragonRunSettings.py | 1 + 3 files changed, 25 insertions(+), 16 deletions(-) diff --git a/smartsim/_core/launcher/dragon/dragonBackend.py b/smartsim/_core/launcher/dragon/dragonBackend.py index 2e528092f..2fda87646 100644 --- a/smartsim/_core/launcher/dragon/dragonBackend.py +++ b/smartsim/_core/launcher/dragon/dragonBackend.py @@ -194,6 +194,17 @@ def __init__(self, pid: int) -> None: self._infra_ddict: t.Optional[dragon_ddict.DDict] = None self._prioritizer = NodePrioritizer(self._nodes, self._queue_lock) + self._nodes: t.List["dragon_machine.Node"] = [] + """Node capability information for hosts in the allocation""" + self._hosts: t.List[str] = [] + """List of hosts available in allocation""" + self._cpus: t.List[int] = [] + """List of cpu-count by node""" + self._gpus: t.List[int] = [] + """List of gpu-count by node""" + self._allocated_hosts: t.Dict[str, t.Set[str]] = {} + """Mapping with hostnames as keys and a set of running step IDs as the value""" + @property def hosts(self) -> list[str]: with self._queue_lock: @@ -229,16 +240,10 @@ def _initialize_hosts(self) -> None: self._nodes = [ dragon_machine.Node(node) for node in dragon_machine.System().nodes ] - self._hosts: t.List[str] = sorted(node.hostname for node in self._nodes) - """List of hosts available in allocation""" + self._hosts = sorted(node.hostname for node in self._nodes) self._cpus = [node.num_cpus for node in self._nodes] - """List of cpu-count by node""" self._gpus = [node.num_gpus for node in self._nodes] - """List of gpu-count by node""" - self._allocated_hosts: t.Dict[str, t.Set[str]] = collections.defaultdict( - set - ) - """Mapping with hostnames as keys and a set of executing step IDs as the value""" + self._allocated_hosts = collections.defaultdict(set) def __str__(self) -> str: return self.status_message diff --git a/smartsim/_core/launcher/dragon/pqueue.py b/smartsim/_core/launcher/dragon/pqueue.py index d160174e6..a9faf76b1 100644 --- a/smartsim/_core/launcher/dragon/pqueue.py +++ b/smartsim/_core/launcher/dragon/pqueue.py @@ -124,7 +124,8 @@ def add( reference counted. :param tracking_id: a unique task identifier executing on the node - to add""" + to add + :raises ValueError: if tracking_id is already assigned to this node""" if tracking_id in self.assigned_tasks: raise ValueError("Attempted adding task more than once") @@ -140,7 +141,8 @@ def remove( """Update the reference counter to indicate the removal of a process. :param tracking_id: a unique task identifier executing on the node - to remove""" + to remove + :raises ValueError: if tracking_id is already assigned to this node""" if tracking_id and tracking_id not in self.assigned_tasks: raise ValueError("Attempted removal of untracked item") @@ -335,12 +337,12 @@ def _check_satisfiable_n( return True - def _get_next_available_node( + def _get_next_unassigned_node( self, heap: t.List[_TrackedNode], tracking_id: t.Optional[str] = None, ) -> t.Optional[Node]: - """Finds the next node w/the least amount of running processes and + """Finds the next node with no running processes and ensures that any elements that were directly updated are updated in the priority structure before being made available @@ -402,7 +404,7 @@ def _get_next_n_available_nodes( return next_nodes while len(next_nodes) < num_items: - if next_node := self._get_next_available_node(heap, tracking_id): + if next_node := self._get_next_unassigned_node(heap, tracking_id): next_nodes.append(next_node) continue break @@ -430,7 +432,7 @@ def next( tracking_id: t.Optional[str] = None, hosts: t.Optional[t.List[str]] = None, ) -> t.Optional[Node]: - """Find the next available node using the supplied filter to target + """Find the next unsassigned node using the supplied filter to target a specific node capability :param filter_on: the subset of nodes to query for available nodes @@ -456,9 +458,10 @@ def next_n( :param filter_on: the subset of nodes to query for available nodes :param tracking_id: unique task identifier to track :param hosts: a list of hostnames used to filter the available nodes - :returns: Collection of reserved nodes""" + :returns: Collection of reserved nodes + :raises ValueError: if the hosts parameter is an empty list""" if hosts is not None and not hosts: - raise ValueError("No host names provided") + raise ValueError("No hostnames provided") heap = self._create_sub_heap(hosts, filter_on) return self._get_next_n_available_nodes(num_items, heap, tracking_id) diff --git a/smartsim/settings/dragonRunSettings.py b/smartsim/settings/dragonRunSettings.py index a003bd7d8..15e585544 100644 --- a/smartsim/settings/dragonRunSettings.py +++ b/smartsim/settings/dragonRunSettings.py @@ -100,6 +100,7 @@ def set_hostlist(self, host_list: t.Union[str, t.List[str]]) -> None: """Specify the hostlist for this job :param host_list: hosts to launch on + :raises ValueError: if an empty host list is supplied """ if not host_list: raise ValueError("empty hostlist provided") From a585d54ee7886c5ae3d32cbfd83241f74459e2ce Mon Sep 17 00:00:00 2001 From: ankona <3595025+ankona@users.noreply.github.com> Date: Mon, 19 Aug 2024 16:49:05 -0500 Subject: [PATCH 24/24] fix message change breaking a test assertion --- tests/test_node_prioritizer.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/test_node_prioritizer.py b/tests/test_node_prioritizer.py index 70df5a840..abb4624b6 100644 --- a/tests/test_node_prioritizer.py +++ b/tests/test_node_prioritizer.py @@ -479,7 +479,7 @@ def test_node_prioritizer_empty_subheap_next_w_hosts() -> None: with pytest.raises(ValueError) as ex: p.next(hosts=hostnames) - assert "No host names provided" == ex.value.args[0] + assert "No hostnames provided" == ex.value.args[0] def test_node_prioritizer_empty_subheap_next_n_w_hosts() -> None: @@ -504,7 +504,7 @@ def test_node_prioritizer_empty_subheap_next_n_w_hosts() -> None: with pytest.raises(ValueError) as ex: p.next_n(num_requested, hosts=hostnames) - assert "No host names provided" == ex.value.args[0] + assert "No hostnames provided" == ex.value.args[0] @pytest.mark.parametrize("num_requested", [-100, -1, 0])