From c77f71d2562efef7c4a49912af1c625b157098f1 Mon Sep 17 00:00:00 2001 From: Wenjun Si <wenjun.swj@alibaba-inc.com> Date: Fri, 3 Dec 2021 14:15:21 +0800 Subject: [PATCH] PARTIAL add worker scheduling --- .../oscar/backends/ray/tests/test_ray_pool.py | 2 + mars/oscar/backends/test/pool.py | 2 + mars/oscar/core.pyx | 4 +- mars/remote/core.py | 2 + mars/services/cluster/api/oscar.py | 2 +- mars/services/core.py | 39 ++ mars/services/scheduling/api/oscar.py | 77 ++- .../scheduling/supervisor/__init__.py | 2 - .../scheduling/supervisor/autoscale.py | 21 +- .../scheduling/supervisor/globalslot.py | 144 ----- .../services/scheduling/supervisor/manager.py | 386 ++++++------ .../scheduling/supervisor/queueing.py | 307 ---------- .../services/scheduling/supervisor/service.py | 45 +- .../supervisor/tests/test_globalslot.py | 76 --- .../supervisor/tests/test_manager.py | 207 ++++--- .../supervisor/tests/test_queue_balance.py | 414 ++++++------- .../supervisor/tests/test_queueing.py | 137 ----- .../services/scheduling/tests/test_service.py | 22 +- mars/services/scheduling/worker/__init__.py | 12 +- .../scheduling/worker/exec/__init__.py | 16 + mars/services/scheduling/worker/exec/actor.py | 575 ++++++++++++++++++ mars/services/scheduling/worker/exec/core.py | 69 +++ .../scheduling/worker/exec/prepare.py | 244 ++++++++ .../scheduling/worker/exec/tests/__init__.py | 13 + .../scheduling/worker/exec/tests/test_exec.py | 546 +++++++++++++++++ .../worker/exec/tests/test_prepare.py | 13 + mars/services/scheduling/worker/execution.py | 478 --------------- mars/services/scheduling/worker/queues.py | 184 ++++++ mars/services/scheduling/worker/quota.py | 33 +- mars/services/scheduling/worker/service.py | 35 +- .../services/scheduling/worker/slotmanager.py | 52 ++ .../scheduling/worker/tests/test_execution.py | 493 --------------- .../scheduling/worker/tests/test_quota.py | 14 +- .../worker/tests/test_workerslot.py | 346 ----------- mars/services/scheduling/worker/workerslot.py | 313 ---------- mars/services/storage/api/oscar.py | 7 +- mars/services/subtask/api.py | 16 +- mars/services/subtask/core.py | 2 + mars/services/subtask/worker/processor.py | 61 +- mars/services/subtask/worker/runner.py | 23 +- mars/services/task/api/oscar.py | 10 + mars/services/task/supervisor/manager.py | 1 + mars/services/task/supervisor/stage.py | 19 +- mars/services/tests/fault_injection_patch.py | 2 +- mars/utils.py | 6 +- 45 files changed, 2527 insertions(+), 2945 deletions(-) delete mode 100644 mars/services/scheduling/supervisor/globalslot.py delete mode 100644 mars/services/scheduling/supervisor/queueing.py delete mode 100644 mars/services/scheduling/supervisor/tests/test_globalslot.py delete mode 100644 mars/services/scheduling/supervisor/tests/test_queueing.py create mode 100644 mars/services/scheduling/worker/exec/__init__.py create mode 100644 mars/services/scheduling/worker/exec/actor.py create mode 100644 mars/services/scheduling/worker/exec/core.py create mode 100644 mars/services/scheduling/worker/exec/prepare.py create mode 100644 mars/services/scheduling/worker/exec/tests/__init__.py create mode 100644 mars/services/scheduling/worker/exec/tests/test_exec.py create mode 100644 mars/services/scheduling/worker/exec/tests/test_prepare.py delete mode 100644 mars/services/scheduling/worker/execution.py create mode 100644 mars/services/scheduling/worker/queues.py create mode 100644 mars/services/scheduling/worker/slotmanager.py delete mode 100644 mars/services/scheduling/worker/tests/test_execution.py delete mode 100644 mars/services/scheduling/worker/tests/test_workerslot.py delete mode 100644 mars/services/scheduling/worker/workerslot.py diff --git a/mars/oscar/backends/ray/tests/test_ray_pool.py b/mars/oscar/backends/ray/tests/test_ray_pool.py index c37fa28be7..641b558585 100644 --- a/mars/oscar/backends/ray/tests/test_ray_pool.py +++ b/mars/oscar/backends/ray/tests/test_ray_pool.py @@ -30,6 +30,8 @@ class TestActor(mo.Actor): + __test__ = False + async def kill(self, address, uid): actor_ref = await mo.actor_ref(address, uid) task = asyncio.create_task(actor_ref.crash()) diff --git a/mars/oscar/backends/test/pool.py b/mars/oscar/backends/test/pool.py index 07ce7aac63..bd8bdafa01 100644 --- a/mars/oscar/backends/test/pool.py +++ b/mars/oscar/backends/test/pool.py @@ -23,6 +23,8 @@ class TestMainActorPool(MainActorPool): + __test__ = False + @classmethod def get_external_addresses( cls, address: str, n_process: int = None, ports: List[int] = None diff --git a/mars/oscar/core.pyx b/mars/oscar/core.pyx index 91dfb4d151..8fdc48e09c 100644 --- a/mars/oscar/core.pyx +++ b/mars/oscar/core.pyx @@ -111,8 +111,8 @@ cdef class ActorRefMethod: """ Wrapper for an Actor method at client """ - cdef ActorRef ref - cdef object method_name + cdef public ActorRef ref + cdef public object method_name cdef object _options def __init__(self, ref, method_name, options=None): diff --git a/mars/remote/core.py b/mars/remote/core.py index ca7276c238..23df45768b 100644 --- a/mars/remote/core.py +++ b/mars/remote/core.py @@ -57,6 +57,8 @@ def __init__( n_output=None, **kw, ): + function_args = function_args or [] + function_kwargs = function_kwargs or {} super().__init__( _function=function, _function_args=function_args, diff --git a/mars/services/cluster/api/oscar.py b/mars/services/cluster/api/oscar.py index a943280b95..89b64da8b5 100644 --- a/mars/services/cluster/api/oscar.py +++ b/mars/services/cluster/api/oscar.py @@ -211,7 +211,7 @@ async def get_mars_versions(self) -> List[str]: node_info_ref = await self._get_node_info_ref() return await node_info_ref.get_mars_versions() - async def get_bands(self) -> Dict: + async def get_bands(self) -> Dict[BandType, int]: """ Get bands that can be used for computation on current node. diff --git a/mars/services/core.py b/mars/services/core.py index 42334bb4f6..743ab8bffc 100644 --- a/mars/services/core.py +++ b/mars/services/core.py @@ -20,6 +20,16 @@ import warnings from typing import Dict, Iterable, List, Union +from .. import oscar as mo +from ..lib.aio import alru_cache +from ..serialization.serializables import ( + Serializable, + BytesField, + StringField, + TupleField, + DictField, +) + _ModulesType = Union[List, str, None] @@ -28,6 +38,35 @@ class NodeRole(enum.Enum): WORKER = 1 +class ActorCallback(Serializable): + actor_uid: bytes = BytesField("actor_uid") + actor_address: str = StringField("actor_address") + actor_method: str = StringField("actor_method") + args: tuple = TupleField("args") + kwargs: dict = DictField("kwargs") + + def __init__(self, ref_method=None, **kw): + if ref_method is not None: + kw["actor_uid"] = ref_method.ref.uid + kw["actor_address"] = ref_method.ref.address + kw["actor_method"] = ref_method.method_name + kw["args"] = kw.get("args") or () + kw["kwargs"] = kw.get("kwargs") or {} + super().__init__(**kw) + + @classmethod + @alru_cache(cache_exceptions=False) + async def _get_ref(cls, actor_uid: bytes, actor_address: str): + return await mo.actor_ref(actor_uid, address=actor_address) + + async def __call__(self, *args, **kwargs): + ref = await self._get_ref(self.actor_uid, self.actor_address) + args = self.args + args + kw = self.kwargs.copy() + kw.update(kwargs) + return await getattr(ref, self.actor_method)(*args, **kw) + + class AbstractService(abc.ABC): _instances = dict() diff --git a/mars/services/scheduling/api/oscar.py b/mars/services/scheduling/api/oscar.py index 067f8c1438..e2d5862d64 100644 --- a/mars/services/scheduling/api/oscar.py +++ b/mars/services/scheduling/api/oscar.py @@ -24,14 +24,11 @@ class SchedulingAPI(AbstractSchedulingAPI): - def __init__( - self, session_id: str, address: str, manager_ref=None, queueing_ref=None - ): + def __init__(self, session_id: str, address: str, manager_ref=None): self._session_id = session_id self._address = address self._manager_ref = manager_ref - self._queueing_ref = queueing_ref @classmethod @alru_cache @@ -41,13 +38,8 @@ async def create(cls: Type[APIType], session_id: str, address: str) -> APIType: manager_ref = await mo.actor_ref( SubtaskManagerActor.gen_uid(session_id), address=address ) - from ..supervisor.queueing import SubtaskQueueingActor - - queueing_ref = await mo.actor_ref( - SubtaskQueueingActor.gen_uid(session_id), address=address - ) - scheduling_api = SchedulingAPI(session_id, address, manager_ref, queueing_ref) + scheduling_api = SchedulingAPI(session_id, address, manager_ref) return scheduling_api async def get_subtask_schedule_summaries( @@ -55,6 +47,21 @@ async def get_subtask_schedule_summaries( ) -> List[SubtaskScheduleSummary]: return await self._manager_ref.get_schedule_summaries(task_id) + async def cache_subtasks( + self, subtasks: List[Subtask], priorities: Optional[List[Tuple]] = None + ): + """ + Add subtask graph to cache for fast forwarding + + Parameters + ---------- + subtasks + list of subtasks to be submitted to service + priorities + list of priorities of subtasks + """ + await self._manager_ref.cache_subtasks(subtasks, priorities) + async def add_subtasks( self, subtasks: List[Subtask], priorities: Optional[List[Tuple]] = None ): @@ -88,12 +95,12 @@ async def update_subtask_priority(self, subtask_id: str, priority: Tuple): @update_subtask_priority.batch async def update_subtask_priority(self, args_list, kwargs_list): - await self._queueing_ref.update_subtask_priority.batch( - *( - self._queueing_ref.update_subtask_priority.delay(*args, **kwargs) - for args, kwargs in zip(args_list, kwargs_list) - ) - ) + subtask_ids, priorities = [], [] + for args, kwargs in zip(args_list, kwargs_list): + subtask_id, priority = self.update_subtask_priority.bind(*args, **kwargs) + subtask_ids.append(subtask_id) + priorities.append(priority) + await self._manager_ref.update_subtask_priorities(subtask_ids, priorities) async def cancel_subtasks( self, subtask_ids: List[str], kill_timeout: Union[float, int] = 5 @@ -128,33 +135,39 @@ async def finish_subtasks(self, subtask_ids: List[str], schedule_next: bool = Tr class MockSchedulingAPI(SchedulingAPI): @classmethod async def create(cls: Type[APIType], session_id: str, address: str) -> APIType: - from ..supervisor import GlobalSlotManagerActor, AutoscalerActor - - await mo.create_actor( - GlobalSlotManagerActor, - uid=GlobalSlotManagerActor.default_uid(), - address=address, - ) - await mo.create_actor( - AutoscalerActor, {}, uid=AutoscalerActor.default_uid(), address=address - ) + # from ..supervisor import AutoscalerActor + # await mo.create_actor( + # AutoscalerActor, {}, uid=AutoscalerActor.default_uid(), address=address + # ) from .... import resource as mars_resource from ..worker import ( SubtaskExecutionActor, - WorkerSlotManagerActor, + SubtaskPrepareQueueActor, + SubtaskExecutionQueueActor, WorkerQuotaManagerActor, + SlotManagerActor, ) await mo.create_actor( - SubtaskExecutionActor, - subtask_max_retries=0, - uid=SubtaskExecutionActor.default_uid(), + SlotManagerActor, + uid=SlotManagerActor.default_uid(), address=address, ) await mo.create_actor( - WorkerSlotManagerActor, - uid=WorkerSlotManagerActor.default_uid(), + SubtaskPrepareQueueActor, + uid=SubtaskPrepareQueueActor.default_uid(), + address=address, + ) + await mo.create_actor( + SubtaskExecutionQueueActor, + uid=SubtaskExecutionQueueActor.default_uid(), + address=address, + ) + await mo.create_actor( + SubtaskExecutionActor, + subtask_max_retries=0, + uid=SubtaskExecutionActor.default_uid(), address=address, ) await mo.create_actor( diff --git a/mars/services/scheduling/supervisor/__init__.py b/mars/services/scheduling/supervisor/__init__.py index e3b68d8717..fc27315a68 100644 --- a/mars/services/scheduling/supervisor/__init__.py +++ b/mars/services/scheduling/supervisor/__init__.py @@ -14,7 +14,5 @@ from .assigner import AssignerActor from .autoscale import AutoscalerActor -from .globalslot import GlobalSlotManagerActor from .manager import SubtaskManagerActor -from .queueing import SubtaskQueueingActor from .service import SchedulingSupervisorService diff --git a/mars/services/scheduling/supervisor/autoscale.py b/mars/services/scheduling/supervisor/autoscale.py index b5157b4022..28bf2bee0f 100644 --- a/mars/services/scheduling/supervisor/autoscale.py +++ b/mars/services/scheduling/supervisor/autoscale.py @@ -46,11 +46,11 @@ async def __post_create__(self): strategy_cls = getattr(importlib.import_module(module), name) else: strategy_cls = PendingTaskBacklogStrategy - from ..supervisor import GlobalSlotManagerActor - - self.global_slot_ref = await mo.actor_ref( - GlobalSlotManagerActor.default_uid(), address=self.address - ) + # from ..supervisor import GlobalSlotManagerActor + # + # self.global_slot_ref = await mo.actor_ref( + # GlobalSlotManagerActor.default_uid(), address=self.address + # ) self._cluster_api = await ClusterAPI.create(self.address) self._strategy = await strategy_cls.create(self._autoscale_conf, self) if self._enabled: @@ -62,11 +62,12 @@ async def __pre_destroy__(self): await self._strategy.stop() async def register_session(self, session_id: str, address: str): - from .queueing import SubtaskQueueingActor - - self.queueing_refs[session_id] = await mo.actor_ref( - SubtaskQueueingActor.gen_uid(session_id), address=address - ) + pass + # from .queueing import SubtaskQueueingActor + # + # self.queueing_refs[session_id] = await mo.actor_ref( + # SubtaskQueueingActor.gen_uid(session_id), address=address + # ) async def unregister_session(self, session_id: str): self.queueing_refs.pop(session_id, None) diff --git a/mars/services/scheduling/supervisor/globalslot.py b/mars/services/scheduling/supervisor/globalslot.py deleted file mode 100644 index 06ab62a44c..0000000000 --- a/mars/services/scheduling/supervisor/globalslot.py +++ /dev/null @@ -1,144 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import asyncio -import logging -import time -from collections import defaultdict -from typing import List, DefaultDict, Dict, Tuple - -from .... import oscar as mo -from ....typing import BandType - -logger = logging.getLogger(__name__) - - -class GlobalSlotManagerActor(mo.Actor): - # {(address, resource_type): {(session_id, subtask_id): slot_id}} - _band_stid_slots: DefaultDict[BandType, Dict[Tuple[str, str], int]] - _band_used_slots: DefaultDict[BandType, int] - _band_total_slots: Dict[BandType, int] - - def __init__(self): - self._band_stid_slots = defaultdict(dict) - self._band_used_slots = defaultdict(lambda: 0) - self._band_idle_start_time = dict() - self._band_idle_events = dict() - self._band_total_slots = dict() - - self._cluster_api = None - - self._band_watch_task = None - - async def __post_create__(self): - from ...cluster.api import ClusterAPI - - self._cluster_api = await ClusterAPI.create(self.address) - - async def watch_bands(): - async for bands in self._cluster_api.watch_all_bands(): - old_bands = set(self._band_total_slots.keys()) - self._band_total_slots = bands - new_bands = set(bands.keys()) - old_bands - for band in new_bands: - self._update_slot_usage(band, 0) - - self._band_watch_task = asyncio.create_task(watch_bands()) - - async def __pre_destroy__(self): - self._band_watch_task.cancel() - - async def refresh_bands(self): - self._band_total_slots = await self._cluster_api.get_all_bands() - - @mo.extensible - async def apply_subtask_slots( - self, - band: BandType, - session_id: str, - subtask_ids: List[str], - subtask_slots: List[int], - ) -> List[str]: - if not self._band_total_slots or band not in self._band_total_slots: - self._band_total_slots = await self._cluster_api.get_all_bands() - - idx = 0 - # only ready bands will pass - if band in self._band_total_slots: - total_slots = self._band_total_slots[band] - for stid, slots in zip(subtask_ids, subtask_slots): - if self._band_used_slots[band] + slots > total_slots: - break - self._band_stid_slots[band][(session_id, stid)] = slots - self._update_slot_usage(band, slots) - idx += 1 - if idx == 0: - logger.debug( - "No slots available, status: %r, request: %r", - self._band_used_slots, - subtask_slots, - ) - return subtask_ids[:idx] - - @mo.extensible - def update_subtask_slots( - self, band: BandType, session_id: str, subtask_id: str, slots: int - ): - session_subtask_id = (session_id, subtask_id) - subtask_slots = self._band_stid_slots[band] - - if session_subtask_id not in subtask_slots: - return - - slots_delta = slots - subtask_slots[session_subtask_id] - subtask_slots[session_subtask_id] = slots - self._update_slot_usage(band, slots_delta) - - @mo.extensible - def release_subtask_slots(self, band: BandType, session_id: str, subtask_id: str): - # todo ensure slots released when subtasks ends in all means - slots_delta = self._band_stid_slots[band].pop((session_id, subtask_id), 0) - self._update_slot_usage(band, -slots_delta) - - def _update_slot_usage(self, band: BandType, slots_usage_delta: float): - self._band_used_slots[band] += slots_usage_delta - if self._band_used_slots[band] == 0: - self._band_used_slots.pop(band) - self._band_idle_start_time[band] = time.time() - if band in self._band_idle_events: - self._band_idle_events.pop(band).set() - else: - self._band_idle_start_time[band] = -1 - - def get_used_slots(self) -> Dict[BandType, int]: - return self._band_used_slots - - async def get_idle_bands(self, idle_duration: int): - """Return a band list which all bands has been idle for at least `idle_duration` seconds.""" - now = time.time() - idle_bands = [] - for band in self._band_total_slots.keys(): - idle_start_time = self._band_idle_start_time[band] - if idle_start_time > 0 and now >= idle_start_time + idle_duration: - idle_bands.append(band) - return idle_bands - - async def wait_band_idle(self, band: BandType): - if self._band_idle_start_time[band] <= 0: - if band in self._band_idle_events: - event = self._band_idle_events[band] - else: - event = asyncio.Event() - self._band_idle_events[band] = event - return event.wait() diff --git a/mars/services/scheduling/supervisor/manager.py b/mars/services/scheduling/supervisor/manager.py index 3fb48fd576..149b5fd035 100644 --- a/mars/services/scheduling/supervisor/manager.py +++ b/mars/services/scheduling/supervisor/manager.py @@ -13,12 +13,16 @@ # limitations under the License. import asyncio +import functools import logging +import operator from collections import defaultdict from dataclasses import dataclass, field -from typing import Dict, List, Optional, Tuple, Union +from typing import Dict, List, Mapping, Optional, Set, Tuple, Union +from weakref import WeakValueDictionary from .... import oscar as mo +from ....core.operand import Fetch from ....lib.aio import alru_cache from ....oscar.backends.message import ProfilingContext from ....oscar.errors import MarsError @@ -39,24 +43,32 @@ @dataslots @dataclass class SubtaskScheduleInfo: + __slots__ = ("__weakref__",) + subtask: Subtask - band_futures: Dict[BandType, asyncio.Future] = field(default_factory=dict) + priority: Tuple max_reschedules: int = 0 num_reschedules: int = 0 + cached_workers: Set[str] = field(default_factory=set) + submitted_bands: Set[BandType] = field(default_factory=set) def to_summary(self, **kwargs) -> SubtaskScheduleSummary: return SubtaskScheduleSummary( task_id=self.subtask.task_id, subtask_id=self.subtask.subtask_id, - bands=list(self.band_futures.keys()), + bands=list(self.submitted_bands), num_reschedules=self.num_reschedules, **kwargs, ) class SubtaskManagerActor(mo.Actor): - _subtask_infos: Dict[str, SubtaskScheduleInfo] # subtask id -> schedule info - _subtask_summaries: Dict[str, SubtaskScheduleSummary] # subtask id -> summary + # subtask id -> schedule info + _subtask_infos: Dict[str, SubtaskScheduleInfo] + # subtask id -> summary + _subtask_summaries: Dict[str, SubtaskScheduleSummary] + # chunk key -> schedule info + _chunk_key_to_subtask_info: Mapping[str, SubtaskScheduleInfo] @classmethod def gen_uid(cls, session_id: str): @@ -70,45 +82,56 @@ def __init__( self._session_id = session_id self._subtask_infos = dict() self._subtask_summaries = dict() + self._chunk_key_to_subtask_info = WeakValueDictionary() self._subtask_max_reschedules = subtask_max_reschedules - self._queueing_ref = None + self._assigner_ref = None self._global_slot_ref = None async def __post_create__(self): - from .queueing import SubtaskQueueingActor - - self._queueing_ref = await mo.actor_ref( - SubtaskQueueingActor.gen_uid(self._session_id), address=self.address - ) - from ..supervisor import GlobalSlotManagerActor + from .assigner import AssignerActor - self._global_slot_ref = await mo.actor_ref( - GlobalSlotManagerActor.default_uid(), address=self.address + self._assigner_ref = await mo.actor_ref( + AssignerActor.gen_uid(self._session_id), address=self.address ) @alru_cache async def _get_task_api(self): return await TaskAPI.create(self._session_id, self.address) - async def add_subtasks(self, subtasks: List[Subtask], priorities: List[Tuple]): - async with redirect_subtask_errors(self, subtasks): - for subtask in subtasks: - # the extra_config may be None. the extra config overwrites the default value. - subtask_max_reschedules = ( - subtask.extra_config.get("subtask_max_reschedules") - if subtask.extra_config - else None - ) - if subtask_max_reschedules is None: - subtask_max_reschedules = self._subtask_max_reschedules - self._subtask_infos[subtask.subtask_id] = SubtaskScheduleInfo( - subtask, max_reschedules=subtask_max_reschedules - ) + def _put_subtask_with_priority(self, subtask: Subtask, priority: Tuple = None): + # if already placed, just update priority and return + if subtask.subtask_id in self._subtask_infos: + self._subtask_infos[subtask.subtask_id].priority = priority + return + + # the extra_config may be None. the extra config overwrites the default value. + subtask_max_reschedules = ( + subtask.extra_config.get("subtask_max_reschedules") + if subtask.extra_config + else None + ) + if subtask_max_reschedules is None: + subtask_max_reschedules = self._subtask_max_reschedules + subtask_info = self._subtask_infos[subtask.subtask_id] = SubtaskScheduleInfo( + subtask, priority, max_reschedules=subtask_max_reschedules + ) + + if subtask.chunk_graph: + for result_chunk in subtask.chunk_graph.results: + self._chunk_key_to_subtask_info[result_chunk.key] = subtask_info - virtual_subtasks = [subtask for subtask in subtasks if subtask.virtual] - for subtask in virtual_subtasks: - task_api = await self._get_task_api() + async def _filter_virtual_subtasks( + self, subtasks: List[Subtask], priorities: List[Tuple] + ): + # filter out virtual subtasks + has_virtual = False + task_api = None + for subtask, priority in zip(subtasks, priorities): + if subtask.virtual: + if task_api is None: + task_api = await self._get_task_api() + has_virtual = True await task_api.set_subtask_result( SubtaskResult( subtask_id=subtask.subtask_id, @@ -118,17 +141,140 @@ async def add_subtasks(self, subtasks: List[Subtask], priorities: List[Tuple]): status=SubtaskStatus.succeeded, ) ) + if not has_virtual: + return subtasks, priorities + else: + subtasks_new, priorities_new = [], [] + for subtask, priority in zip(subtasks, priorities): + if subtask.virtual: + continue + subtasks_new.append(subtask) + priorities_new.append(priority) + return subtasks_new, priorities_new + + async def add_subtasks(self, subtasks: List[Subtask], priorities: List[Tuple]): + # filter out virtual subtasks + subtasks, priorities = await self._filter_virtual_subtasks(subtasks, priorities) + async with redirect_subtask_errors(self, subtasks): + for subtask, priority in zip(subtasks, priorities): + self._put_subtask_with_priority(subtask, priority) + + band_list = await self._assigner_ref.assign_subtasks(subtasks) + + band_to_subtasks = defaultdict(list) + band_to_priorities = defaultdict(list) + + for subtask, priority, band in zip(subtasks, priorities, band_list): + info = self._subtask_infos[subtask.subtask_id] + if band[0] not in info.cached_workers: + subtask_data = subtask + else: + subtask_data = subtask.subtask_id + band_to_subtasks[band].append(subtask_data) + band_to_priorities[band].append(priority) - await self._queueing_ref.add_subtasks( - [subtask for subtask in subtasks if not subtask.virtual], priorities + coros = [] + for band in band_to_subtasks.keys(): + execution_ref = await self._get_execution_ref(band[0]) + coro = execution_ref.submit_subtasks( + band_to_subtasks[band], + band_to_priorities[band], + self.address, + band[1], + ) + coros.append(coro) + + yield tuple(coros) + + for subtask, band in zip(subtasks, band_list): + self._subtask_infos[subtask.subtask_id].submitted_bands.add(band) + + async def cache_subtasks( + self, subtasks: List[Subtask], priorities: Optional[List[Tuple]] = None + ): + band_to_subtasks = defaultdict(list) + band_to_priorities = defaultdict(list) + subtask_id_to_workers = dict() + for subtask, priority in zip(subtasks, priorities): + if subtask.virtual: + continue + self._put_subtask_with_priority(subtask, priority) + + bands_list = [] + if subtask.chunk_graph: + for chunk in subtask.chunk_graph: + if not isinstance(chunk.op, Fetch): + continue + try: + src_subtask_info = self._chunk_key_to_subtask_info[chunk.key] + except KeyError: + continue + bands_list.append(src_subtask_info.submitted_bands) + + if not bands_list: + cache_bands = [] + else: + cache_bands = list(functools.reduce(operator.and_, bands_list)) + + for band in cache_bands: + band_to_subtasks[band].append(subtask) + band_to_priorities[band].append(priority) + + subtask_id_to_workers[subtask.subtask_id] = set( + band[0] for band in cache_bands + ) + + coros = [] + for band in band_to_subtasks.keys(): + execution_ref = await self._get_execution_ref(band[0]) + coro = execution_ref.cache_subtasks( + band_to_subtasks[band], + band_to_priorities[band], + self.address, + band[1], ) - await self._queueing_ref.submit_subtasks.tell() + coros.append(coro) + yield tuple(coros) + + for subtask, priority in zip(subtasks, priorities): + if subtask.virtual: + continue + self._subtask_infos[ + subtask.subtask_id + ].cached_workers = subtask_id_to_workers[subtask.subtask_id] + + async def update_subtask_priorities( + self, subtask_ids: List[str], priorities: List[Tuple] = None + ): + worker_to_subtask_ids = defaultdict(list) + worker_to_priorities = defaultdict(list) + for subtask_id, priority in zip(subtask_ids, priorities): + try: + info = self._subtask_infos[subtask_id] + except KeyError: + continue + info.priority = priority + workers = set(info.cached_workers) | set( + band[0] for band in info.submitted_bands + ) + for worker in workers: + worker_to_subtask_ids[worker].append(subtask_id) + worker_to_priorities[worker].append(priority) + coros = [] + for worker in worker_to_subtask_ids.keys(): + ref = await self._get_execution_ref(worker) + coro = ref.update_subtask_priorities( + worker_to_subtask_ids[worker], + worker_to_priorities[worker], + ) + coros.append(coro) + yield tuple(coros) @alru_cache(maxsize=10000) - async def _get_execution_ref(self, band: BandType): - from ..worker.execution import SubtaskExecutionActor + async def _get_execution_ref(self, address: str): + from ..worker.exec import SubtaskExecutionActor - return await mo.actor_ref(SubtaskExecutionActor.default_uid(), address=band[0]) + return await mo.actor_ref(SubtaskExecutionActor.default_uid(), address=address) async def finish_subtasks(self, subtask_ids: List[str], schedule_next: bool = True): band_tasks = defaultdict(lambda: 0) @@ -139,17 +285,9 @@ async def finish_subtasks(self, subtask_ids: List[str], schedule_next: bool = Tr is_finished=True ) if schedule_next: - for band in subtask_info.band_futures.keys(): + for band in subtask_info.submitted_bands: band_tasks[band] += 1 - if band_tasks: - coros = [] - for band, subtask_count in band_tasks.items(): - coros.append( - self._queueing_ref.submit_subtasks.tell(band, subtask_count) - ) - await asyncio.wait(coros) - def _get_subtasks_by_ids(self, subtask_ids: List[str]) -> List[Optional[Subtask]]: subtasks = [] for stid in subtask_ids: @@ -159,140 +297,46 @@ def _get_subtasks_by_ids(self, subtask_ids: List[str]) -> List[Optional[Subtask] subtasks.append(None) return subtasks - async def submit_subtask_to_band(self, subtask_id: str, band: BandType): - async with redirect_subtask_errors( - self, self._get_subtasks_by_ids([subtask_id]) - ): - try: - subtask_info = self._subtask_infos[subtask_id] - execution_ref = await self._get_execution_ref(band) - extra_config = subtask_info.subtask.extra_config - profiling_context = ( - ProfilingContext(subtask_info.subtask.task_id) - if extra_config and extra_config.get("enable_profiling") - else None - ) - task = asyncio.create_task( - execution_ref.run_subtask.options( - profiling_context=profiling_context - ).send(subtask_info.subtask, band[1], self.address) - ) - subtask_info.band_futures[band] = task - result = yield task - task_api = await self._get_task_api() - await task_api.set_subtask_result(result) - except (OSError, MarsError) as ex: - # TODO: We should handle ServerClosed Error. - if ( - subtask_info.subtask.retryable - and subtask_info.num_reschedules < subtask_info.max_reschedules - ): - logger.error( - "Reschedule subtask %s due to %s", - subtask_info.subtask.subtask_id, - ex, - ) - subtask_info.num_reschedules += 1 - await self._queueing_ref.add_subtasks( - [subtask_info.subtask], - [subtask_info.subtask.priority or tuple()], - ) - await self._queueing_ref.submit_subtasks.tell() - else: - raise ex - except asyncio.CancelledError: - raise - except Exception as ex: - if ( - subtask_info.subtask.retryable - and subtask_info.num_reschedules < subtask_info.max_reschedules - ): - logger.error( - "Failed to reschedule subtask %s, " - "num_reschedules: %s, max_reschedules: %s, unhandled exception: %s", - subtask_info.subtask.subtask_id, - subtask_info.num_reschedules, - subtask_info.max_reschedules, - ex, - ) - raise ex - finally: - # make sure slot is released before marking tasks as finished - await self._global_slot_ref.release_subtask_slots( - band, - subtask_info.subtask.session_id, - subtask_info.subtask.subtask_id, - ) - logger.debug( - "Slot released for band %s after subtask %s", - band, - subtask_info.subtask.subtask_id, - ) - async def cancel_subtasks( self, subtask_ids: List[str], kill_timeout: Union[float, int] = 5 ): - queued_subtask_ids = [] - single_cancel_tasks = [] - task_api = await self._get_task_api() - async def cancel_single_task(subtask, raw_tasks, cancel_tasks): - if cancel_tasks: - await asyncio.wait(cancel_tasks) - if raw_tasks: - dones, _ = await asyncio.wait(raw_tasks) - else: - dones = [] - if not dones or all(fut.cancelled() for fut in dones): - await task_api.set_subtask_result( - SubtaskResult( - subtask_id=subtask.subtask_id, - session_id=subtask.session_id, - task_id=subtask.task_id, - status=SubtaskStatus.cancelled, - ) + async def cancel_and_wait(worker: str, subtask_ids: List[str]): + execution_ref = await self._get_execution_ref(worker) + await execution_ref.cancel_subtasks(subtask_ids, kill_timeout=kill_timeout) + await execution_ref.wait_subtasks(subtask_ids) + + result_batches = [] + for subtask_id in subtask_ids: + subtask = self._subtask_infos[subtask_id].subtask + result = SubtaskResult( + subtask_id=subtask.subtask_id, + session_id=subtask.session_id, + task_id=subtask.task_id, + status=SubtaskStatus.cancelled, ) + result_batches.append(task_api.set_subtask_result.delay(result)) + await task_api.set_subtask_result.batch(*result_batches) + + worker_to_subtask_ids = defaultdict(list) for subtask_id in subtask_ids: - if subtask_id not in self._subtask_infos: - # subtask may already finished or not submitted at all + try: + info = self._subtask_infos[subtask_id] + except KeyError: continue - - subtask_info = self._subtask_infos[subtask_id] - raw_tasks_to_cancel = list(subtask_info.band_futures.values()) - - if not subtask_info.band_futures: - queued_subtask_ids.append(subtask_id) - single_cancel_tasks.append( - asyncio.create_task( - cancel_single_task( - subtask_info.subtask, raw_tasks_to_cancel, [] - ) - ) - ) - else: - cancel_tasks = [] - for band in subtask_info.band_futures.keys(): - execution_ref = await self._get_execution_ref(band) - cancel_tasks.append( - asyncio.create_task( - execution_ref.cancel_subtask( - subtask_id, kill_timeout=kill_timeout - ) - ) - ) - single_cancel_tasks.append( - asyncio.create_task( - cancel_single_task( - subtask_info.subtask, raw_tasks_to_cancel, cancel_tasks - ) - ) - ) - if queued_subtask_ids: - await self._queueing_ref.remove_queued_subtasks(queued_subtask_ids) - if single_cancel_tasks: - yield asyncio.wait(single_cancel_tasks) + workers = set(band[0] for band in info.submitted_bands) | set( + info.cached_workers + ) + for worker in workers: + worker_to_subtask_ids[worker].append(subtask_id) + + cancel_coros = [] + for worker, subtask_ids in worker_to_subtask_ids.items(): + cancel_coros.append(cancel_and_wait(worker, subtask_ids)) + if cancel_coros: + await asyncio.gather(*cancel_coros) for subtask_id in subtask_ids: subtask_info = self._subtask_infos.pop(subtask_id, None) diff --git a/mars/services/scheduling/supervisor/queueing.py b/mars/services/scheduling/supervisor/queueing.py deleted file mode 100644 index 2b401bce71..0000000000 --- a/mars/services/scheduling/supervisor/queueing.py +++ /dev/null @@ -1,307 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import asyncio -import copy -import heapq -import logging -from collections import defaultdict -from dataclasses import dataclass -from typing import DefaultDict, Dict, List, Optional, Tuple, Union - -from .... import oscar as mo -from ....lib.aio import alru_cache -from ....utils import dataslots -from ...subtask import Subtask -from ...task import TaskAPI -from ..utils import redirect_subtask_errors - -logger = logging.getLogger(__name__) - -_DEFAULT_SUBMIT_PERIOD = 0 - - -@dataslots -@dataclass -class HeapItem: - subtask: Subtask - priority: Tuple - - def __lt__(self, other: "HeapItem"): - return self.priority > other.priority - - -class SubtaskQueueingActor(mo.Actor): - _stid_to_bands: DefaultDict[str, List[Tuple]] - _stid_to_items: Dict[str, HeapItem] - _band_queues: DefaultDict[Tuple, List[HeapItem]] - - @classmethod - def gen_uid(cls, session_id: str): - return f"{session_id}_subtask_queueing" - - def __init__(self, session_id: str, submit_period: Union[float, int] = None): - self._session_id = session_id - self._stid_to_bands = defaultdict(list) - self._stid_to_items = dict() - # Note that we need to ensure top item in every band heap queue is valid, - # so that we can ensure band queue is busy if the band queue is not empty. - self._band_queues = defaultdict(list) - - self._cluster_api = None - self._slots_ref = None - self._assigner_ref = None - - self._band_slot_nums = dict() - self._band_watch_task = None - self._max_enqueue_id = 0 - - self._periodical_submit_task = None - self._submit_period = submit_period or _DEFAULT_SUBMIT_PERIOD - - async def __post_create__(self): - from ...cluster import ClusterAPI - - self._cluster_api = await ClusterAPI.create(self.address) - self._band_slot_nums = {} - - async def watch_bands(): - async for bands in self._cluster_api.watch_all_bands(): - # confirm ready bands indeed changed - if bands != self._band_slot_nums: - old_band_slot_nums = self._band_slot_nums - self._band_slot_nums = copy.deepcopy(bands) - if self._band_queues: - await self.balance_queued_subtasks() - # Refresh global slot manager to get latest bands, - # so that subtasks reassigned to the new bands can be - # ensured to get submitted as least one subtask every band - # successfully. - await self._slots_ref.refresh_bands() - all_bands = {*bands.keys(), *old_band_slot_nums.keys()} - bands_delta = {} - for b in all_bands: - delta = bands.get(b, 0) - old_band_slot_nums.get(b, 0) - if delta != 0: - bands_delta[b] = delta - # Submit tasks on new bands manually, otherwise some subtasks - # will never got submitted. Note that we must ensure every new - # band will get at least one subtask submitted successfully. - # Later subtasks submit on the band will be triggered by the - # success of previous subtasks on the same band. - logger.info( - "Bands changed with delta %s, submit all bands.", - bands_delta, - ) - await self.ref().submit_subtasks() - - self._band_watch_task = asyncio.create_task(watch_bands()) - - from .globalslot import GlobalSlotManagerActor - - [self._slots_ref] = await self._cluster_api.get_supervisor_refs( - [GlobalSlotManagerActor.default_uid()] - ) - from .assigner import AssignerActor - - self._assigner_ref = await mo.actor_ref( - AssignerActor.gen_uid(self._session_id), address=self.address - ) - - if self._submit_period > 0: - self._periodical_submit_task = self.ref().periodical_submit.tell_delay( - delay=self._submit_period - ) - - async def __pre_destroy__(self): - self._band_watch_task.cancel() - if self._periodical_submit_task is not None: # pragma: no branch - self._periodical_submit_task.cancel() - - async def periodical_submit(self): - await self.ref().submit_subtasks.tell() - self._periodical_submit_task = self.ref().periodical_submit.tell_delay( - delay=self._submit_period - ) - - @alru_cache - async def _get_task_api(self): - return await TaskAPI.create(self._session_id, self.address) - - @alru_cache(cache_exceptions=False) - async def _get_manager_ref(self): - from .manager import SubtaskManagerActor - - return await mo.actor_ref( - SubtaskManagerActor.gen_uid(self._session_id), address=self.address - ) - - async def add_subtasks(self, subtasks: List[Subtask], priorities: List[Tuple]): - bands = await self._assigner_ref.assign_subtasks(subtasks) - for subtask, band, priority in zip(subtasks, bands, priorities): - assert band is not None - self._stid_to_bands[subtask.subtask_id].append(band) - heap_item = self._stid_to_items[subtask.subtask_id] = HeapItem( - subtask, priority + (self._max_enqueue_id,) - ) - self._max_enqueue_id += 1 - heapq.heappush(self._band_queues[band], heap_item) - logger.debug("%d subtasks enqueued", len(subtasks)) - - async def submit_subtasks(self, band: Tuple = None, limit: Optional[int] = None): - logger.debug("Submitting subtasks with limit %s", limit) - - if not limit and band not in self._band_slot_nums: - self._band_slot_nums = await self._cluster_api.get_all_bands() - - bands = [band] if band is not None else list(self._band_slot_nums.keys()) - submit_aio_tasks = [] - manager_ref = await self._get_manager_ref() - - apply_delays = [] - submit_items_list = [] - submitted_bands = [] - - for band in bands: - band_limit = limit or self._band_slot_nums[band] - task_queue = self._band_queues[band] - submit_items = dict() - while task_queue and len(submit_items) < band_limit: - self._ensure_top_item_valid(task_queue) - item = heapq.heappop(task_queue) - submit_items[item.subtask.subtask_id] = item - - subtask_ids = list(submit_items) - if not subtask_ids: - continue - - submitted_bands.append(band) - submit_items_list.append(submit_items) - - # todo it is possible to provide slot data with more accuracy - subtask_slots = [1] * len(subtask_ids) - - apply_delays.append( - self._slots_ref.apply_subtask_slots.delay( - band, self._session_id, subtask_ids, subtask_slots - ) - ) - - async with redirect_subtask_errors( - self, - [ - item.subtask - for submit_items in submit_items_list - for item in submit_items.values() - ], - ): - submitted_ids_list = await self._slots_ref.apply_subtask_slots.batch( - *apply_delays - ) - - for band, submit_items, submitted_ids in zip( - submitted_bands, submit_items_list, submitted_ids_list - ): - subtask_ids = list(submit_items) - task_queue = self._band_queues[band] - - async with redirect_subtask_errors( - self, [item.subtask for item in submit_items.values()] - ): - non_submitted_ids = [k for k in submit_items if k not in submitted_ids] - if submitted_ids: - for stid in subtask_ids: - if stid not in submitted_ids: - continue - item = submit_items[stid] - logger.debug( - "Submit subtask %s to band %r", - item.subtask.subtask_id, - band, - ) - submit_aio_tasks.append( - asyncio.create_task( - manager_ref.submit_subtask_to_band.tell( - item.subtask.subtask_id, band - ) - ) - ) - await asyncio.sleep(0) - self.remove_queued_subtasks([item.subtask.subtask_id]) - else: - logger.debug("No slots available") - - for stid in non_submitted_ids: - heapq.heappush(task_queue, submit_items[stid]) - - if submit_aio_tasks: - yield asyncio.gather(*submit_aio_tasks) - - def _ensure_top_item_valid(self, task_queue): - """Clean invalid subtask item from queue to ensure that""" - while ( - task_queue and task_queue[0].subtask.subtask_id not in self._stid_to_items - ): - # skip removed items (as they may be re-pushed into the queue) - heapq.heappop(task_queue) - - @mo.extensible - def update_subtask_priority(self, subtask_id: str, priority: Tuple): - if subtask_id not in self._stid_to_bands: - return - for band in self._stid_to_bands[subtask_id]: - new_item = HeapItem(self._stid_to_items[subtask_id].subtask, priority) - self._stid_to_items[subtask_id] = new_item - heapq.heappush(self._band_queues[band], new_item) - - def remove_queued_subtasks(self, subtask_ids: List[str]): - for stid in subtask_ids: - bands = self._stid_to_bands.pop(stid, []) - self._stid_to_items.pop(stid, None) - for band in bands: - band_queue = self._band_queues.get(band) - self._ensure_top_item_valid(band_queue) - - async def all_bands_busy(self) -> bool: - """Return True if all bands queue has tasks waiting to be submitted.""" - bands = set(self._band_slot_nums.keys()) - if set(self._band_queues.keys()).issuperset(bands): - return all(len(self._band_queues[band]) > 0 for band in bands) - return False - - async def balance_queued_subtasks(self): - # record length of band queues - band_num_queued_subtasks = { - band: len(queue) for band, queue in self._band_queues.items() - } - move_queued_subtasks = await self._assigner_ref.reassign_subtasks( - band_num_queued_subtasks - ) - items = [] - # rewrite band queues according to feedbacks from assigner - for band, move in move_queued_subtasks.items(): - task_queue = self._band_queues[band] - assert move + len(task_queue) >= 0 - for _ in range(abs(move)): - if move < 0: - # TODO: pop item of low priority - item = heapq.heappop(task_queue) - self._stid_to_bands[item.subtask.subtask_id].remove(band) - items.append(item) - elif move > 0: - item = items.pop() - self._stid_to_bands[item.subtask.subtask_id].append(band) - heapq.heappush(task_queue, item) - if len(task_queue) == 0: - self._band_queues.pop(band) diff --git a/mars/services/scheduling/supervisor/service.py b/mars/services/scheduling/supervisor/service.py index 5c2c96b8c9..e40e449ba6 100644 --- a/mars/services/scheduling/supervisor/service.py +++ b/mars/services/scheduling/supervisor/service.py @@ -42,14 +42,6 @@ class SchedulingSupervisorService(AbstractService): """ async def start(self): - from .globalslot import GlobalSlotManagerActor - - await mo.create_actor( - GlobalSlotManagerActor, - uid=GlobalSlotManagerActor.default_uid(), - address=self._address, - ) - autoscale_config = self._config.get("scheduling", {}).get("autoscale", {}) await mo.create_actor( AutoscalerActor, @@ -67,14 +59,6 @@ async def stop(self): ) ) - from .globalslot import GlobalSlotManagerActor - - await mo.destroy_actor( - mo.create_actor_ref( - uid=GlobalSlotManagerActor.default_uid(), address=self._address - ) - ) - async def create_session(self, session_id: str): service_config = self._config or dict() scheduling_config = service_config.get("scheduling", {}) @@ -84,25 +68,13 @@ async def create_session(self, session_id: str): from .assigner import AssignerActor - assigner_coro = mo.create_actor( + await mo.create_actor( AssignerActor, session_id, address=self._address, uid=AssignerActor.gen_uid(session_id), ) - from .queueing import SubtaskQueueingActor - - queueing_coro = mo.create_actor( - SubtaskQueueingActor, - session_id, - scheduling_config.get("submit_period"), - address=self._address, - uid=SubtaskQueueingActor.gen_uid(session_id), - ) - - await asyncio.gather(assigner_coro, queueing_coro) - from .manager import SubtaskManagerActor await mo.create_actor( @@ -113,15 +85,14 @@ async def create_session(self, session_id: str): uid=SubtaskManagerActor.gen_uid(session_id), ) - from .autoscale import AutoscalerActor - - autoscaler_ref = await mo.actor_ref( - AutoscalerActor.default_uid(), address=self._address - ) - await autoscaler_ref.register_session(session_id, self._address) + # from .autoscale import AutoscalerActor + # + # autoscaler_ref = await mo.actor_ref( + # AutoscalerActor.default_uid(), address=self._address + # ) + # await autoscaler_ref.register_session(session_id, self._address) async def destroy_session(self, session_id: str): - from .queueing import SubtaskQueueingActor from .manager import SubtaskManagerActor from .assigner import AssignerActor from .autoscale import AutoscalerActor @@ -132,7 +103,7 @@ async def destroy_session(self, session_id: str): await autoscaler_ref.unregister_session(session_id) destroy_tasks = [] - for actor_cls in [SubtaskManagerActor, SubtaskQueueingActor, AssignerActor]: + for actor_cls in [SubtaskManagerActor, AssignerActor]: ref = await mo.actor_ref( actor_cls.gen_uid(session_id), address=self._address ) diff --git a/mars/services/scheduling/supervisor/tests/test_globalslot.py b/mars/services/scheduling/supervisor/tests/test_globalslot.py deleted file mode 100644 index 92252b36ab..0000000000 --- a/mars/services/scheduling/supervisor/tests/test_globalslot.py +++ /dev/null @@ -1,76 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import asyncio - -import pytest - -from ..... import oscar as mo -from ....cluster import ClusterAPI, MockClusterAPI -from ....session import MockSessionAPI -from ...supervisor import GlobalSlotManagerActor - - -@pytest.fixture -async def actor_pool(): - pool = await mo.create_actor_pool("127.0.0.1", n_process=0) - - async with pool: - session_id = "test_session" - await MockClusterAPI.create(pool.external_address) - await MockSessionAPI.create(pool.external_address, session_id=session_id) - - global_slot_ref = await mo.create_actor( - GlobalSlotManagerActor, - uid=GlobalSlotManagerActor.default_uid(), - address=pool.external_address, - ) - - try: - yield pool, session_id, global_slot_ref - finally: - await mo.destroy_actor(global_slot_ref) - await MockClusterAPI.cleanup(pool.external_address) - - -@pytest.mark.asyncio -async def test_global_slot(actor_pool): - pool, session_id, global_slot_ref = actor_pool - - cluster_api = await ClusterAPI.create(pool.external_address) - bands = await cluster_api.get_all_bands() - band = (pool.external_address, "numa-0") - band_slots = bands[band] - - assert band in await global_slot_ref.get_idle_bands(0) - assert ["subtask0"] == await global_slot_ref.apply_subtask_slots( - band, session_id, ["subtask0"], [1] - ) - assert band not in await global_slot_ref.get_idle_bands(0) - - await global_slot_ref.update_subtask_slots(band, session_id, "subtask0", band_slots) - assert [] == await global_slot_ref.apply_subtask_slots( - band, session_id, ["subtask1"], [1] - ) - - wait_coro = global_slot_ref.wait_band_idle(band) - (done, pending) = await asyncio.wait([wait_coro], timeout=0.5) - assert not done - await global_slot_ref.release_subtask_slots(band, session_id, "subtask0") - (done, pending) = await asyncio.wait([wait_coro], timeout=0.5) - assert done - assert band in await global_slot_ref.get_idle_bands(0) - assert ["subtask1"] == await global_slot_ref.apply_subtask_slots( - band, session_id, ["subtask1"], [1] - ) diff --git a/mars/services/scheduling/supervisor/tests/test_manager.py b/mars/services/scheduling/supervisor/tests/test_manager.py index 14c35960a9..9a13d6fb66 100644 --- a/mars/services/scheduling/supervisor/tests/test_manager.py +++ b/mars/services/scheduling/supervisor/tests/test_manager.py @@ -13,24 +13,23 @@ # limitations under the License. import asyncio -from collections import defaultdict -from typing import List, Tuple +from typing import Dict, List, Optional, Tuple, Union import pytest from ..... import oscar as mo -from .....typing import BandType from ....cluster import MockClusterAPI from ....subtask import Subtask, SubtaskResult, SubtaskStatus from ....task.supervisor.manager import TaskManagerActor -from ...supervisor import ( - SubtaskQueueingActor, - SubtaskManagerActor, - GlobalSlotManagerActor, -) +from ...supervisor import SubtaskManagerActor, AssignerActor from ...worker import SubtaskExecutionActor +class MockAssignerActor(mo.Actor): + async def assign_subtasks(self, subtasks: List[Subtask]): + return [(self.address, "numa-0")] * len(subtasks) + + class MockTaskManagerActor(mo.Actor): def __init__(self): self._results = dict() @@ -42,54 +41,82 @@ def get_result(self, subtask_id: str) -> SubtaskResult: return self._results[subtask_id] -class MockSubtaskQueueingActor(mo.Actor): - def __init__(self): - self._subtasks = dict() - self._error = None - - def add_subtasks(self, subtasks: List[Subtask], priorities: List[Tuple]): - if self._error is not None: - raise self._error - for subtask, priority in zip(subtasks, priorities): - self._subtasks[subtask.subtask_id] = (subtask, priority) - - def submit_subtasks(self, band: BandType, limit: int): - pass - - def remove_queued_subtasks(self, subtask_ids: List[str]): - for stid in subtask_ids: - self._subtasks.pop(stid) - - def set_error(self, error): - self._error = error - - class MockSubtaskExecutionActor(mo.StatelessActor): - def __init__(self): - self._subtask_aiotasks = defaultdict(dict) - self._run_subtask_events = {} + _subtask_aiotasks: Dict[str, asyncio.Task] - async def set_run_subtask_event(self, subtask_id, event): - self._run_subtask_events[subtask_id] = event - - async def run_subtask( - self, subtask: Subtask, band_name: str, supervisor_address: str + def __init__(self): + self._subtask_caches = dict() + self._subtasks = dict() + self._subtask_aiotasks = dict() + self._subtask_submit_events = dict() + + async def set_submit_subtask_event(self, subtask_id: str, event: asyncio.Event): + self._subtask_submit_events[subtask_id] = event + + async def cache_subtasks( + self, + subtasks: List[Subtask], + priorities: List[Tuple], + supervisor_address: str, + band_name: str, ): - self._run_subtask_events[subtask.subtask_id].set() - task = self._subtask_aiotasks[subtask.subtask_id][ - band_name - ] = asyncio.create_task(asyncio.sleep(20)) - return await task - - def cancel_subtask(self, subtask_id: str, kill_timeout: int = 5): - for task in self._subtask_aiotasks[subtask_id].values(): - task.cancel() + for subtask in subtasks: + self._subtask_caches[subtask.subtask_id] = subtask + + async def submit_subtasks( + self, + subtasks: List[Union[str, Subtask]], + priorities: List[Tuple], + supervisor_address: str, + band_name: str, + ): + for subtask in subtasks: + if isinstance(subtask, str): + subtask = self._subtask_caches[subtask] + self._subtasks[subtask.subtask_id] = subtask + self._subtask_aiotasks[subtask.subtask_id] = asyncio.create_task( + asyncio.sleep(20) + ) + self._subtask_submit_events[subtask.subtask_id].set() + + async def cancel_subtasks( + self, subtask_ids: List[str], kill_timeout: Optional[int] = 5 + ): + for subtask_id in subtask_ids: + self._subtask_aiotasks[subtask_id].cancel() - async def wait_subtask(self, subtask_id: str, band_name: str): - try: - yield self._subtask_aiotasks[subtask_id][band_name] - except asyncio.CancelledError: - pass + async def wait_subtasks(self, subtask_ids: List[str]): + yield asyncio.wait( + [self._subtask_aiotasks[subtask_id] for subtask_id in subtask_ids] + ) + results = [] + for subtask_id in subtask_ids: + subtask = self._subtasks[subtask_id] + aiotask = self._subtask_aiotasks[subtask_id] + if not aiotask.done(): + subtask_kw = dict(status=SubtaskStatus.running) + elif aiotask.cancelled(): + subtask_kw = dict(status=SubtaskStatus.cancelled) + elif aiotask.exception() is not None: + exc = aiotask.exception() + tb = exc.__traceback__ + subtask_kw = dict( + status=SubtaskStatus.errored, + error=exc, + traceback=tb, + ) + else: + subtask_kw = dict(status=SubtaskStatus.succeeded) + results.append( + SubtaskResult( + subtask_id=subtask_id, + session_id=subtask.session_id, + task_id=subtask.task_id, + progress=1.0, + **subtask_kw + ) + ) + raise mo.Return(results) @pytest.fixture @@ -99,16 +126,6 @@ async def actor_pool(): async with pool: session_id = "test_session" await MockClusterAPI.create(pool.external_address) - queue_ref = await mo.create_actor( - MockSubtaskQueueingActor, - uid=SubtaskQueueingActor.gen_uid(session_id), - address=pool.external_address, - ) - slots_ref = await mo.create_actor( - GlobalSlotManagerActor, - uid=GlobalSlotManagerActor.default_uid(), - address=pool.external_address, - ) task_manager_ref = await mo.create_actor( MockTaskManagerActor, uid=TaskManagerActor.gen_uid(session_id), @@ -119,16 +136,20 @@ async def actor_pool(): uid=SubtaskExecutionActor.default_uid(), address=pool.external_address, ) - submitter_ref = await mo.create_actor( + assigner_ref = await mo.create_actor( + MockAssignerActor, + uid=AssignerActor.gen_uid(session_id), + address=pool.external_address, + ) + manager_ref = await mo.create_actor( SubtaskManagerActor, session_id, uid=SubtaskManagerActor.gen_uid(session_id), address=pool.external_address, ) - yield pool, session_id, execution_ref, submitter_ref, queue_ref, task_manager_ref + yield pool, session_id, execution_ref, manager_ref, task_manager_ref - await mo.destroy_actor(slots_ref) await MockClusterAPI.cleanup(pool.external_address) @@ -139,55 +160,33 @@ async def test_subtask_manager(actor_pool): session_id, execution_ref, manager_ref, - queue_ref, task_manager_ref, ) = actor_pool subtask1 = Subtask("subtask1", session_id) subtask2 = Subtask("subtask2", session_id) - await manager_ref.add_subtasks([subtask1, subtask2], [(1,), (2,)]) - run_subtask1_event, run_subtask2_event = asyncio.Event(), asyncio.Event() - await execution_ref.set_run_subtask_event(subtask1.subtask_id, run_subtask1_event) - await execution_ref.set_run_subtask_event(subtask2.subtask_id, run_subtask2_event) - - submit1 = asyncio.create_task( - manager_ref.submit_subtask_to_band( - subtask1.subtask_id, (pool.external_address, "gpu-0") - ) + submit_subtask1_event = asyncio.Event() + submit_subtask2_event = asyncio.Event() + await execution_ref.set_submit_subtask_event( + subtask1.subtask_id, submit_subtask1_event ) - submit2 = asyncio.create_task( - manager_ref.submit_subtask_to_band( - subtask2.subtask_id, (pool.external_address, "gpu-1") - ) + await execution_ref.set_submit_subtask_event( + subtask2.subtask_id, submit_subtask2_event ) - await asyncio.gather(run_subtask1_event.wait(), run_subtask2_event.wait()) + await manager_ref.cache_subtasks([subtask2], [(2,)]) - await manager_ref.cancel_subtasks([subtask1.subtask_id, subtask2.subtask_id]) + await manager_ref.add_subtasks([subtask1, subtask2], [(1,), (2,)]) await asyncio.wait_for( - asyncio.gather( - execution_ref.wait_subtask(subtask1.subtask_id, "gpu-0"), - execution_ref.wait_subtask(subtask2.subtask_id, "gpu-1"), - ), + asyncio.gather(submit_subtask1_event.wait(), submit_subtask2_event.wait()), + timeout=10, + ) + + await manager_ref.cancel_subtasks([subtask1.subtask_id, subtask2.subtask_id]) + results = await asyncio.wait_for( + execution_ref.wait_subtasks([subtask1.subtask_id, subtask2.subtask_id]), timeout=10, ) - with pytest.raises(asyncio.CancelledError): - await submit1 - with pytest.raises(asyncio.CancelledError): - await submit2 - assert ( - await task_manager_ref.get_result(subtask1.subtask_id) - ).status == SubtaskStatus.cancelled - assert ( - await task_manager_ref.get_result(subtask2.subtask_id) - ).status == SubtaskStatus.cancelled - - subtask3 = Subtask("subtask3", session_id) - - await queue_ref.set_error(ValueError()) - await manager_ref.add_subtasks.tell([subtask3], [(3,)]) - await asyncio.sleep(0.1) - subtask3_result = await task_manager_ref.get_result(subtask3.subtask_id) - assert subtask3_result.status == SubtaskStatus.errored - assert isinstance(subtask3_result.error, ValueError) + assert results[0].status == SubtaskStatus.cancelled + assert results[1].status == SubtaskStatus.cancelled diff --git a/mars/services/scheduling/supervisor/tests/test_queue_balance.py b/mars/services/scheduling/supervisor/tests/test_queue_balance.py index 821b95cf99..29a1b7edd4 100644 --- a/mars/services/scheduling/supervisor/tests/test_queue_balance.py +++ b/mars/services/scheduling/supervisor/tests/test_queue_balance.py @@ -12,210 +12,210 @@ # See the License for the specific language governing permissions and # limitations under the License. -import asyncio -import pytest -from typing import Tuple, List - -from ..... import oscar as mo -from ....cluster import ClusterAPI -from ....cluster.core import NodeRole, NodeStatus -from ....cluster.uploader import NodeInfoUploaderActor -from ....cluster.supervisor.locator import SupervisorPeerLocatorActor -from ....cluster.supervisor.node_info import NodeInfoCollectorActor -from ....subtask import Subtask -from ...supervisor import ( - AssignerActor, - SubtaskManagerActor, - SubtaskQueueingActor, - GlobalSlotManagerActor, -) - - -class MockNodeInfoCollectorActor(NodeInfoCollectorActor): - def __init__(self, timeout=None, check_interval=None): - super().__init__(timeout=timeout, check_interval=check_interval) - self.ready_nodes = { - ("address0", "numa-0"): 2, - ("address1", "numa-0"): 2, - ("address2", "numa-0"): 2, - } - - async def update_node_info( - self, address, role, env=None, resource=None, detail=None, status=None - ): - if "address" in address and status == NodeStatus.STOPPING: - del self.ready_nodes[(address, "numa-0")] - await super().update_node_info(address, role, env, resource, detail, status) - - def get_all_bands(self, role=None, statuses=None): - if statuses == {NodeStatus.READY}: - return self.ready_nodes - else: - return { - ("address0", "numa-0"): 2, - ("address1", "numa-0"): 2, - ("address2", "numa-0"): 2, - } - - -class FakeClusterAPI(ClusterAPI): - @classmethod - async def create(cls, address: str, **kw): - dones, _ = await asyncio.wait( - [ - mo.create_actor( - SupervisorPeerLocatorActor, - "fixed", - address, - uid=SupervisorPeerLocatorActor.default_uid(), - address=address, - ), - mo.create_actor( - MockNodeInfoCollectorActor, - uid=NodeInfoCollectorActor.default_uid(), - address=address, - ), - mo.create_actor( - NodeInfoUploaderActor, - NodeRole.WORKER, - interval=kw.get("upload_interval"), - band_to_slots=kw.get("band_to_slots"), - use_gpu=kw.get("use_gpu", False), - uid=NodeInfoUploaderActor.default_uid(), - address=address, - ), - ] - ) - - for task in dones: - try: - task.result() - except mo.ActorAlreadyExist: # pragma: no cover - pass - - api = await super().create(address=address) - await api.mark_node_ready() - return api - - -class MockSlotsActor(mo.Actor): - @mo.extensible - def apply_subtask_slots( - self, - band: Tuple, - session_id: str, - subtask_ids: List[str], - subtask_slots: List[int], - ): - return subtask_ids - - -class MockAssignerActor(mo.Actor): - def assign_subtasks(self, subtasks: List[Subtask]): - return [subtask.expect_bands[0] for subtask in subtasks] - - def reassign_subtasks(self, band_num_queued_subtasks): - if len(band_num_queued_subtasks.keys()) == 1: - [(band, _)] = band_num_queued_subtasks.items() - return {band: 0} - return { - ("address1", "numa-0"): -8, - ("address0", "numa-0"): 0, - ("address2", "numa-0"): 8, - } - - -class MockSubtaskManagerActor(mo.Actor): - def __init__(self): - self._subtask_ids, self._bands = [], [] - - @mo.extensible - def submit_subtask_to_band(self, subtask_id: str, band: Tuple): - self._subtask_ids.append(subtask_id) - self._bands.append(band) - - def dump_data(self): - return self._subtask_ids, self._bands - - -@pytest.fixture -async def actor_pool(): - pool = await mo.create_actor_pool("127.0.0.1", n_process=0) - - async with pool: - session_id = "test_session" - cluster_api = await FakeClusterAPI.create(pool.external_address) - - # create assigner actor - await mo.create_actor( - MockAssignerActor, - uid=AssignerActor.gen_uid(session_id), - address=pool.external_address, - ) - # create queueing actor - manager_ref = await mo.create_actor( - MockSubtaskManagerActor, - uid=SubtaskManagerActor.gen_uid(session_id), - address=pool.external_address, - ) - # create slots actor - slots_ref = await mo.create_actor( - MockSlotsActor, - uid=GlobalSlotManagerActor.default_uid(), - address=pool.external_address, - ) - # create queueing actor - queueing_ref = await mo.create_actor( - SubtaskQueueingActor, - session_id, - 1, - uid=SubtaskQueueingActor.gen_uid(session_id), - address=pool.external_address, - ) - - yield pool, session_id, cluster_api, queueing_ref, slots_ref, manager_ref - - await mo.destroy_actor(queueing_ref) - - -async def _queue_subtasks(num_subtasks, expect_bands, queueing_ref): - if not num_subtasks: - return - subtasks = [Subtask(expect_bands[0] + "-" + str(i)) for i in range(num_subtasks)] - for subtask in subtasks: - subtask.expect_bands = [expect_bands] - priorities = [(i,) for i in range(num_subtasks)] - - await queueing_ref.add_subtasks(subtasks, priorities) - - -@pytest.mark.asyncio -async def test_subtask_queueing(actor_pool): - _pool, session_id, cluster_api, queueing_ref, slots_ref, manager_ref = actor_pool - nums_subtasks = [9, 8, 1] - expects_bands = [ - ("address0", "numa-0"), - ("address1", "numa-0"), - ("address2", "numa-0"), - ] - for num_subtasks, expect_bands in zip(nums_subtasks, expects_bands): - await _queue_subtasks(num_subtasks, expect_bands, queueing_ref) - - await cluster_api.set_node_status( - node="address1", role=NodeRole.WORKER, status=NodeStatus.STOPPING - ) - - # 9 subtasks on ('address0', 'numa-0') - await queueing_ref.submit_subtasks(band=("address0", "numa-0"), limit=10) - commited_subtask_ids, _commited_bands = await manager_ref.dump_data() - assert len(commited_subtask_ids) == 9 - - # 0 subtasks on ('address1', 'numa-0') - await queueing_ref.submit_subtasks(band=("address1", "numa-0"), limit=10) - commited_subtask_ids, _commited_bands = await manager_ref.dump_data() - assert len(commited_subtask_ids) == 9 - - # 9 subtasks on ('address2', 'numa-0') - await queueing_ref.submit_subtasks(band=("address2", "numa-0"), limit=10) - commited_subtask_ids, _commited_bands = await manager_ref.dump_data() - assert len(commited_subtask_ids) == 18 +# import asyncio +# import pytest +# from typing import Tuple, List +# +# from ..... import oscar as mo +# from ....cluster import ClusterAPI +# from ....cluster.core import NodeRole, NodeStatus +# from ....cluster.uploader import NodeInfoUploaderActor +# from ....cluster.supervisor.locator import SupervisorPeerLocatorActor +# from ....cluster.supervisor.node_info import NodeInfoCollectorActor +# from ....subtask import Subtask +# from ...supervisor import ( +# AssignerActor, +# SubtaskManagerActor, +# SubtaskQueueingActor, +# GlobalSlotManagerActor, +# ) +# +# +# class MockNodeInfoCollectorActor(NodeInfoCollectorActor): +# def __init__(self, timeout=None, check_interval=None): +# super().__init__(timeout=timeout, check_interval=check_interval) +# self.ready_nodes = { +# ("address0", "numa-0"): 2, +# ("address1", "numa-0"): 2, +# ("address2", "numa-0"): 2, +# } +# +# async def update_node_info( +# self, address, role, env=None, resource=None, detail=None, status=None +# ): +# if "address" in address and status == NodeStatus.STOPPING: +# del self.ready_nodes[(address, "numa-0")] +# await super().update_node_info(address, role, env, resource, detail, status) +# +# def get_all_bands(self, role=None, statuses=None): +# if statuses == {NodeStatus.READY}: +# return self.ready_nodes +# else: +# return { +# ("address0", "numa-0"): 2, +# ("address1", "numa-0"): 2, +# ("address2", "numa-0"): 2, +# } +# +# +# class FakeClusterAPI(ClusterAPI): +# @classmethod +# async def create(cls, address: str, **kw): +# dones, _ = await asyncio.wait( +# [ +# mo.create_actor( +# SupervisorPeerLocatorActor, +# "fixed", +# address, +# uid=SupervisorPeerLocatorActor.default_uid(), +# address=address, +# ), +# mo.create_actor( +# MockNodeInfoCollectorActor, +# uid=NodeInfoCollectorActor.default_uid(), +# address=address, +# ), +# mo.create_actor( +# NodeInfoUploaderActor, +# NodeRole.WORKER, +# interval=kw.get("upload_interval"), +# band_to_slots=kw.get("band_to_slots"), +# use_gpu=kw.get("use_gpu", False), +# uid=NodeInfoUploaderActor.default_uid(), +# address=address, +# ), +# ] +# ) +# +# for task in dones: +# try: +# task.result() +# except mo.ActorAlreadyExist: # pragma: no cover +# pass +# +# api = await super().create(address=address) +# await api.mark_node_ready() +# return api +# +# +# class MockSlotsActor(mo.Actor): +# @mo.extensible +# def apply_subtask_slots( +# self, +# band: Tuple, +# session_id: str, +# subtask_ids: List[str], +# subtask_slots: List[int], +# ): +# return subtask_ids +# +# +# class MockAssignerActor(mo.Actor): +# def assign_subtasks(self, subtasks: List[Subtask]): +# return [subtask.expect_bands[0] for subtask in subtasks] +# +# def reassign_subtasks(self, band_num_queued_subtasks): +# if len(band_num_queued_subtasks.keys()) == 1: +# [(band, _)] = band_num_queued_subtasks.items() +# return {band: 0} +# return { +# ("address1", "numa-0"): -8, +# ("address0", "numa-0"): 0, +# ("address2", "numa-0"): 8, +# } +# +# +# class MockSubtaskManagerActor(mo.Actor): +# def __init__(self): +# self._subtask_ids, self._bands = [], [] +# +# @mo.extensible +# def submit_subtask_to_band(self, subtask_id: str, band: Tuple): +# self._subtask_ids.append(subtask_id) +# self._bands.append(band) +# +# def dump_data(self): +# return self._subtask_ids, self._bands +# +# +# @pytest.fixture +# async def actor_pool(): +# pool = await mo.create_actor_pool("127.0.0.1", n_process=0) +# +# async with pool: +# session_id = "test_session" +# cluster_api = await FakeClusterAPI.create(pool.external_address) +# +# # create assigner actor +# await mo.create_actor( +# MockAssignerActor, +# uid=AssignerActor.gen_uid(session_id), +# address=pool.external_address, +# ) +# # create queueing actor +# manager_ref = await mo.create_actor( +# MockSubtaskManagerActor, +# uid=SubtaskManagerActor.gen_uid(session_id), +# address=pool.external_address, +# ) +# # create slots actor +# slots_ref = await mo.create_actor( +# MockSlotsActor, +# uid=GlobalSlotManagerActor.default_uid(), +# address=pool.external_address, +# ) +# # create queueing actor +# queueing_ref = await mo.create_actor( +# SubtaskQueueingActor, +# session_id, +# 1, +# uid=SubtaskQueueingActor.gen_uid(session_id), +# address=pool.external_address, +# ) +# +# yield pool, session_id, cluster_api, queueing_ref, slots_ref, manager_ref +# +# await mo.destroy_actor(queueing_ref) +# +# +# async def _queue_subtasks(num_subtasks, expect_bands, queueing_ref): +# if not num_subtasks: +# return +# subtasks = [Subtask(expect_bands[0] + "-" + str(i)) for i in range(num_subtasks)] +# for subtask in subtasks: +# subtask.expect_bands = [expect_bands] +# priorities = [(i,) for i in range(num_subtasks)] +# +# await queueing_ref.add_subtasks(subtasks, priorities) +# +# +# @pytest.mark.asyncio +# async def test_subtask_queueing(actor_pool): +# _pool, session_id, cluster_api, queueing_ref, slots_ref, manager_ref = actor_pool +# nums_subtasks = [9, 8, 1] +# expects_bands = [ +# ("address0", "numa-0"), +# ("address1", "numa-0"), +# ("address2", "numa-0"), +# ] +# for num_subtasks, expect_bands in zip(nums_subtasks, expects_bands): +# await _queue_subtasks(num_subtasks, expect_bands, queueing_ref) +# +# await cluster_api.set_node_status( +# node="address1", role=NodeRole.WORKER, status=NodeStatus.STOPPING +# ) +# +# # 9 subtasks on ('address0', 'numa-0') +# await queueing_ref.submit_subtasks(band=("address0", "numa-0"), limit=10) +# commited_subtask_ids, _commited_bands = await manager_ref.dump_data() +# assert len(commited_subtask_ids) == 9 +# +# # 0 subtasks on ('address1', 'numa-0') +# await queueing_ref.submit_subtasks(band=("address1", "numa-0"), limit=10) +# commited_subtask_ids, _commited_bands = await manager_ref.dump_data() +# assert len(commited_subtask_ids) == 9 +# +# # 9 subtasks on ('address2', 'numa-0') +# await queueing_ref.submit_subtasks(band=("address2", "numa-0"), limit=10) +# commited_subtask_ids, _commited_bands = await manager_ref.dump_data() +# assert len(commited_subtask_ids) == 18 diff --git a/mars/services/scheduling/supervisor/tests/test_queueing.py b/mars/services/scheduling/supervisor/tests/test_queueing.py deleted file mode 100644 index 55904d7b63..0000000000 --- a/mars/services/scheduling/supervisor/tests/test_queueing.py +++ /dev/null @@ -1,137 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import pytest -from typing import Tuple, List - -from ..... import oscar as mo -from ....cluster import MockClusterAPI -from ....subtask import Subtask -from ...supervisor import ( - AssignerActor, - SubtaskManagerActor, - SubtaskQueueingActor, - GlobalSlotManagerActor, -) - - -class MockSlotsActor(mo.Actor): - def __init__(self): - self._capacity = -1 - - def set_capacity(self, capacity: int): - self._capacity = capacity - - @mo.extensible - def apply_subtask_slots( - self, - band: Tuple, - session_id: str, - subtask_ids: List[str], - subtask_slots: List[int], - ): - idx = ( - min(self._capacity, len(subtask_ids)) - if self._capacity >= 0 - else len(subtask_ids) - ) - return subtask_ids[:idx] - - -class MockAssignerActor(mo.Actor): - def assign_subtasks(self, subtasks: List[Subtask]): - return [(self.address, "numa-0")] * len(subtasks) - - -class MockSubtaskManagerActor(mo.Actor): - def __init__(self): - self._subtask_ids, self._bands = [], [] - - @mo.extensible - def submit_subtask_to_band(self, subtask_id: str, band: Tuple): - self._subtask_ids.append(subtask_id) - self._bands.append(band) - - def dump_data(self): - return self._subtask_ids, self._bands - - -@pytest.fixture -async def actor_pool(): - pool = await mo.create_actor_pool("127.0.0.1", n_process=0) - - async with pool: - session_id = "test_session" - await MockClusterAPI.create(pool.external_address) - - # create assigner actor - await mo.create_actor( - MockAssignerActor, - uid=AssignerActor.gen_uid(session_id), - address=pool.external_address, - ) - # create queueing actor - manager_ref = await mo.create_actor( - MockSubtaskManagerActor, - uid=SubtaskManagerActor.gen_uid(session_id), - address=pool.external_address, - ) - # create slots actor - slots_ref = await mo.create_actor( - MockSlotsActor, - uid=GlobalSlotManagerActor.default_uid(), - address=pool.external_address, - ) - # create queueing actor - queueing_ref = await mo.create_actor( - SubtaskQueueingActor, - session_id, - uid=SubtaskQueueingActor.gen_uid(session_id), - address=pool.external_address, - ) - try: - yield pool, session_id, queueing_ref, slots_ref, manager_ref - finally: - await mo.destroy_actor(queueing_ref) - await MockClusterAPI.cleanup(pool.external_address) - - -@pytest.mark.asyncio -async def test_subtask_queueing(actor_pool): - _pool, session_id, queueing_ref, slots_ref, manager_ref = actor_pool - await slots_ref.set_capacity(2) - - subtasks = [Subtask(str(i)) for i in range(5)] - priorities = [(i,) for i in range(5)] - - await queueing_ref.add_subtasks(subtasks, priorities) - # queue: [4 3 2 1 0] - assert await queueing_ref.all_bands_busy() - await queueing_ref.submit_subtasks() - # queue: [2 1 0] - commited_subtask_ids, _commited_bands = await manager_ref.dump_data() - assert commited_subtask_ids == ["4", "3"] - - await queueing_ref.remove_queued_subtasks(["1"]) - # queue: [2 0] - await queueing_ref.update_subtask_priority.batch( - queueing_ref.update_subtask_priority.delay("0", (3,)), - queueing_ref.update_subtask_priority.delay("4", (5,)), - ) - # queue: [0(3) 2] - await queueing_ref.submit_subtasks() - # queue: [] - commited_subtasks, _commited_bands = await manager_ref.dump_data() - assert commited_subtasks == ["4", "3", "0", "2"] - assert not await queueing_ref.all_bands_busy() diff --git a/mars/services/scheduling/tests/test_service.py b/mars/services/scheduling/tests/test_service.py index 5eef78c341..06e0e150c0 100644 --- a/mars/services/scheduling/tests/test_service.py +++ b/mars/services/scheduling/tests/test_service.py @@ -32,7 +32,6 @@ from ...task.supervisor.manager import TaskManagerActor from ...web import WebActor from .. import SchedulingAPI -from ..supervisor import GlobalSlotManagerActor class FakeTaskManager(TaskManagerActor): @@ -41,6 +40,7 @@ def __init__(self, *args, **kwargs): self._events = defaultdict(list) self._results = dict() + @mo.extensible def set_subtask_result(self, subtask_result: SubtaskResult): self._results[subtask_result.subtask_id] = subtask_result for event in self._events[subtask_result.subtask_id]: @@ -160,9 +160,6 @@ async def _get_subtask_summaries_by_web(sv_pool_address, session_id, task_id=Non @pytest.mark.asyncio async def test_schedule_success(actor_pools): sv_pool, worker_pool, session_id, task_manager_ref = actor_pools - global_slot_ref = await mo.actor_ref( - GlobalSlotManagerActor.default_uid(), address=sv_pool.external_address - ) scheduling_api = await SchedulingAPI.create(session_id, sv_pool.external_address) storage_api = await StorageAPI.create(session_id, worker_pool.external_address) @@ -180,8 +177,6 @@ async def test_schedule_success(actor_pools): result = await storage_api.get(result_key) np.testing.assert_array_equal(np.ones((10, 10)) + 1, result) - assert (await global_slot_ref.get_used_slots())["numa-0"] == 0 - [summary] = await _get_subtask_summaries_by_web( sv_pool.external_address, session_id, subtask.task_id ) @@ -192,9 +187,6 @@ async def test_schedule_success(actor_pools): @pytest.mark.asyncio async def test_schedule_queue(actor_pools): sv_pool, worker_pool, session_id, task_manager_ref = actor_pools - global_slot_ref = await mo.actor_ref( - GlobalSlotManagerActor.default_uid(), address=sv_pool.external_address - ) scheduling_api = await SchedulingAPI.create(session_id, sv_pool.external_address) finish_ids, finish_time = [], [] @@ -224,15 +216,10 @@ async def _waiter_fun(subtask_id): await scheduling_api.update_subtask_priority(subtasks[-1].subtask_id, (6,)) await asyncio.gather(*wait_tasks) - assert (await global_slot_ref.get_used_slots())["numa-0"] == 0 - @pytest.mark.asyncio async def test_schedule_error(actor_pools): sv_pool, worker_pool, session_id, task_manager_ref = actor_pools - global_slot_ref = await mo.actor_ref( - GlobalSlotManagerActor.default_uid(), address=sv_pool.external_address - ) scheduling_api = await SchedulingAPI.create(session_id, sv_pool.external_address) def _remote_fun(): @@ -246,15 +233,10 @@ def _remote_fun(): with pytest.raises(ValueError): await task_manager_ref.wait_subtask_result(subtask.subtask_id) - assert (await global_slot_ref.get_used_slots())["numa-0"] == 0 - @pytest.mark.asyncio async def test_schedule_cancel(actor_pools): sv_pool, worker_pool, session_id, task_manager_ref = actor_pools - global_slot_ref = await mo.actor_ref( - GlobalSlotManagerActor.default_uid(), address=sv_pool.external_address - ) scheduling_api = await SchedulingAPI.create(session_id, sv_pool.external_address) def _remote_fun(secs): @@ -293,5 +275,3 @@ async def _waiter_fun(subtask_id): assert all( summary.is_finished and summary.is_cancelled for summary in summaries[2:] ) - - assert (await global_slot_ref.get_used_slots())["numa-0"] == 0 diff --git a/mars/services/scheduling/worker/__init__.py b/mars/services/scheduling/worker/__init__.py index 2df8a8239a..369b0adbfb 100644 --- a/mars/services/scheduling/worker/__init__.py +++ b/mars/services/scheduling/worker/__init__.py @@ -12,11 +12,11 @@ # See the License for the specific language governing permissions and # limitations under the License. -from .execution import SubtaskExecutionActor +from .exec import SubtaskExecutionActor +from .queues import SubtaskExecutionQueueActor, SubtaskPrepareQueueActor from .quota import QuotaActor, MemQuotaActor, WorkerQuotaManagerActor -from .workerslot import ( - WorkerSlotManagerActor, - BandSlotManagerActor, - BandSlotControlActor, -) from .service import SchedulingWorkerService +from .slotmanager import SlotManagerActor + +_ = SubtaskExecutionActor +del _ diff --git a/mars/services/scheduling/worker/exec/__init__.py b/mars/services/scheduling/worker/exec/__init__.py new file mode 100644 index 0000000000..d6091cde2e --- /dev/null +++ b/mars/services/scheduling/worker/exec/__init__.py @@ -0,0 +1,16 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from .actor import SubtaskExecutionActor +from .prepare import SubtaskPreparer diff --git a/mars/services/scheduling/worker/exec/actor.py b/mars/services/scheduling/worker/exec/actor.py new file mode 100644 index 0000000000..d8776a0fac --- /dev/null +++ b/mars/services/scheduling/worker/exec/actor.py @@ -0,0 +1,575 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import asyncio +import functools +import logging +import sys +from typing import Callable, Dict, List, Optional, Tuple, Type, Union + +from ..... import oscar as mo +from .....core.graph import DAG +from .....core.operand import Fetch +from .....lib.aio import alru_cache +from .....oscar.errors import MarsError +from ....cluster import ClusterAPI +from ....core import ActorCallback +from ....subtask import Subtask, SubtaskAPI, SubtaskResult, SubtaskStatus +from ....task import TaskAPI +from ..queues import SubtaskPrepareQueueActor, SubtaskExecutionQueueActor +from ..quota import QuotaActor +from ..slotmanager import SlotManagerActor +from .core import call_with_retry, SubtaskExecutionInfo +from .prepare import SubtaskPreparer, PrepareFastFailed + +logger = logging.getLogger(__name__) + +# the default times to run subtask. +DEFAULT_SUBTASK_MAX_RETRIES = 0 + + +class SubtaskExecutionActor(mo.Actor): + _pred_key_mapping_dag: DAG + _subtask_caches: Dict[str, SubtaskExecutionInfo] + _subtask_executions: Dict[str, SubtaskExecutionInfo] + + _prepare_queue_ref: Union[SubtaskPrepareQueueActor, mo.ActorRef, None] + _execution_queue_ref: Union[SubtaskExecutionQueueActor, mo.ActorRef, None] + _slot_manager_ref: Union[SlotManagerActor, mo.ActorRef, None] + + _subtask_api: SubtaskAPI + _subtask_preparer: SubtaskPreparer + + def __init__( + self, + subtask_max_retries: int = None, + enable_kill_slot: bool = True, + ): + self._pred_key_mapping_dag = DAG() + self._subtask_caches = dict() + self._subtask_executions = dict() + self._prepare_queue_ref = None + self._execution_queue_ref = None + + self._subtask_max_retries = subtask_max_retries or DEFAULT_SUBTASK_MAX_RETRIES + self._enable_kill_slot = enable_kill_slot + + async def __post_create__(self): + self._prepare_queue_ref = await mo.actor_ref( + SubtaskPrepareQueueActor.default_uid(), address=self.address + ) + self._execution_queue_ref = await mo.actor_ref( + SubtaskExecutionQueueActor.default_uid(), address=self.address + ) + self._slot_manager_ref = await mo.actor_ref( + SlotManagerActor.default_uid(), address=self.address + ) + self._subtask_api = await SubtaskAPI.create(self.address) + self._subtask_preparer = SubtaskPreparer(self.address) + + cluster_api = await ClusterAPI.create(self.address) + + self._prepare_process_tasks = dict() + self._execution_process_tasks = dict() + for band in await cluster_api.get_bands(): + self._prepare_process_tasks[band] = asyncio.create_task( + self.handle_prepare_queue(band[1]) + ) + self._execution_process_tasks[band] = asyncio.create_task( + self.handle_execute_queue(band[1]) + ) + + async def __pre_destroy__(self): + for prepare_task in self._prepare_process_tasks.values(): + prepare_task.cancel() + for exec_task in self._execution_process_tasks.values(): + exec_task.cancel() + + @alru_cache(cache_exceptions=False) + async def _get_band_quota_ref( + self, band_name: str + ) -> Union[mo.ActorRef, QuotaActor]: + return await mo.actor_ref(QuotaActor.gen_uid(band_name), address=self.address) + + def _build_subtask_info( + self, + subtask: Subtask, + priority: Tuple, + supervisor_address: str, + band_name: str, + ) -> SubtaskExecutionInfo: + subtask_info = SubtaskExecutionInfo( + subtask, + priority, + supervisor_address=supervisor_address, + band_name=band_name, + max_retries=self._subtask_max_retries, + ) + subtask_info.result = SubtaskResult( + subtask_id=subtask.subtask_id, + session_id=subtask.session_id, + task_id=subtask.task_id, + status=SubtaskStatus.pending, + ) + return subtask_info + + async def cache_subtasks( + self, + subtasks: List[Subtask], + priorities: List[Tuple], + supervisor_address: str, + band_name: str, + ): + for subtask, priority in zip(subtasks, priorities): + if subtask.subtask_id in self._subtask_executions: + continue + self._subtask_caches[subtask.subtask_id] = self._build_subtask_info( + subtask, + priority, + supervisor_address=supervisor_address, + band_name=band_name, + ) + mapping_dag = self._pred_key_mapping_dag + for chunk in subtask.chunk_graph: + if isinstance(chunk.op, Fetch): + mapping_dag.add_node(chunk.key) + mapping_dag.add_node(subtask.subtask_id) + mapping_dag.add_edge(chunk.key, subtask.subtask_id) + + def uncache_subtasks(self, subtask_ids: List[str]): + for subtask_id in subtask_ids: + subtask_info = self._subtask_caches.pop(subtask_id, None) + if subtask_info is None: + continue + subtask = subtask_info.subtask + + try: + self._pred_key_mapping_dag.remove_node(subtask.subtask_id) + except KeyError: + continue + + for chunk in subtask.chunk_graph: + if not isinstance(chunk.op, Fetch): + continue + try: + if self._pred_key_mapping_dag.count_successors(chunk.key) == 0: + self._pred_key_mapping_dag.remove_node(chunk.key) + except KeyError: + continue + + async def update_subtask_priorities( + self, subtask_ids: List[str], priorities: List[Tuple] + ): + assert len(subtask_ids) == len(priorities) + + prepare_delays = [] + execution_delays = [] + for subtask_id, priority in zip(subtask_ids, priorities): + try: + subtask_info = self._subtask_caches[subtask_id] + subtask_info.priority = priority + except KeyError: + pass + + try: + subtask_info = self._subtask_executions[subtask_id] + except KeyError: + continue + if subtask_info.quota_request is not None: + self._execution_queue_ref.update_priority.delay( + subtask_id, subtask_info.band_name, priority + ) + else: + self._prepare_queue_ref.update_priority.delay( + subtask_id, subtask_info.band_name, priority + ) + if prepare_delays: + await self._prepare_queue_ref.update_priority.batch(*prepare_delays) + if execution_delays: + await self._execution_queue_ref.update_priority.batch(*execution_delays) + + async def submit_subtasks( + self, + subtasks: List[Union[str, Subtask]], + priorities: List[Tuple], + supervisor_address: str, + band_name: str, + ): + assert len(subtasks) == len(priorities) + logger.debug("%d subtasks submitted to SubtaskExecutionActor", len(subtasks)) + + put_delays = [] + for subtask, priority in zip(subtasks, priorities): + if isinstance(subtask, str): + try: + subtask = self._subtask_caches[subtask].subtask + except KeyError: + subtask = self._subtask_executions[subtask].subtask + if subtask.subtask_id in self._subtask_executions: + continue + + subtask_info = self._build_subtask_info( + subtask, + priority, + supervisor_address=supervisor_address, + band_name=band_name, + ) + self._subtask_caches.pop(subtask.subtask_id, None) + self._subtask_executions[subtask.subtask_id] = subtask_info + put_delays.append( + self._prepare_queue_ref.put.delay( + subtask.subtask_id, band_name, priority + ) + ) + + if put_delays: + await self._prepare_queue_ref.put.batch(*put_delays) + + async def _dequeue_subtask_ids(self, queue_ref, subtask_ids: List[str]): + removes = [queue_ref.remove.delay(subtask_id) for subtask_id in subtask_ids] + removed_subtask_ids = await queue_ref.remove.batch(*removes) + + infos_to_report = [] + for subtask_id in removed_subtask_ids: + if subtask_id is None: + continue + subtask_info = self._subtask_caches.get(subtask_id) + if subtask_info is None: + subtask_info = self._subtask_executions[subtask_id] + if not subtask_info.finish_future.done(): + self._fill_result_with_exc(subtask_info, exc_cls=asyncio.CancelledError) + infos_to_report.append(subtask_info) + await self._report_subtask_results(infos_to_report) + + @staticmethod + async def _report_subtask_results(subtask_infos: List[SubtaskExecutionInfo]): + if not subtask_infos: + return + task_api = await TaskAPI.create( + subtask_infos[0].result.session_id, subtask_infos[0].supervisor_address + ) + batch = [ + task_api.set_subtask_result.delay(subtask_info.result) + for subtask_info in subtask_infos + ] + await task_api.set_subtask_result.batch(*batch) + + async def cancel_subtasks( + self, subtask_ids: List[str], kill_timeout: Optional[int] = 5 + ): + aio_tasks = [] + for subtask_id in subtask_ids: + try: + subtask_info = self._subtask_executions[subtask_id] + except KeyError: + continue + + subtask_info.kill_timeout = kill_timeout + subtask_info.cancelling = True + aio_tasks.extend(subtask_info.aio_tasks) + for aio_task in subtask_info.aio_tasks: + aio_task.cancel() + + await self._dequeue_subtask_ids(self._prepare_queue_ref, subtask_ids) + await self._dequeue_subtask_ids(self._execution_queue_ref, subtask_ids) + + if aio_tasks: + yield asyncio.wait(aio_tasks) + + self.uncache_subtasks(subtask_ids) + + for subtask_id in subtask_ids: + try: + subtask_info = self._subtask_executions[subtask_id] + except KeyError: + continue + if not subtask_info.result.status.is_done: + self._fill_result_with_exc(subtask_info, exc_cls=asyncio.CancelledError) + + async def wait_subtasks(self, subtask_ids: List[str]): + infos = [ + self._subtask_executions[stid] + for stid in subtask_ids + if stid in self._subtask_executions + ] + if infos: + yield asyncio.wait([info.finish_future for info in infos]) + raise mo.Return([info.result for info in infos]) + + async def handle_prepare_queue(self, band_name: str): + while True: + try: + subtask_id, _ = await self._prepare_queue_ref.get(band_name) + except asyncio.CancelledError: + break + except: # pragma: no cover + logger.exception("Errored when waiting for prepare queue") + continue + + subtask_info = self._subtask_executions[subtask_id] + if subtask_info.cancelling: + continue + + logger.debug(f"Obtained subtask {subtask_id} from prepare queue") + subtask_info.aio_tasks.append( + asyncio.create_task(self._prepare_subtask_with_retry(subtask_info)) + ) + + async def handle_execute_queue(self, band_name: str): + while True: + try: + subtask_id, slot_id = await self._execution_queue_ref.get(band_name) + except asyncio.CancelledError: + break + except: # pragma: no cover + logger.exception("Errored when waiting for execution queue") + continue + + if subtask_id not in self._subtask_executions: + continue + + subtask_info = self._subtask_executions[subtask_id] + if subtask_info.cancelling: + continue + + logger.debug(f"Obtained subtask {subtask_id} from execution queue") + await self._prepare_queue_ref.release_slot( + subtask_info.subtask.subtask_id, errors="ignore" + ) + subtask_info.band_name = band_name + subtask_info.slot_id = slot_id + # if any successors already cached, it must be scheduled + # as soon as possible, thus fast-forward is disabled + subtask_info.forward_successors = any( + c.key in self._pred_key_mapping_dag + for c in subtask_info.subtask.chunk_graph.result_chunks + ) + subtask_info.aio_tasks.append( + asyncio.create_task(self._execute_subtask_with_retry(subtask_info)) + ) + + async def _prepare_subtask_once(self, subtask_info: SubtaskExecutionInfo): + return await self._subtask_preparer.run(subtask_info, fail_fast=False) + + async def _prepare_subtask_with_retry(self, subtask_info: SubtaskExecutionInfo): + try: + await self._call_with_retry(self._prepare_subtask_once, subtask_info) + except: # noqa: E722 # nosec # pylint: disable=bare-except + self._fill_result_with_exc(subtask_info) + await self._report_subtask_results([subtask_info]) + await self._prepare_queue_ref.release_slot( + subtask_info.subtask.subtask_id, errors="ignore" + ) + else: + await self._execution_queue_ref.put( + subtask_info.subtask.subtask_id, + subtask_info.band_name, + subtask_info.priority, + ) + + async def _execute_subtask_once(self, subtask_info: SubtaskExecutionInfo): + subtask = subtask_info.subtask + band_name = subtask_info.band_name + slot_id = subtask_info.slot_id + + finish_callback = ActorCallback( + self.ref().schedule_next, args=(subtask.subtask_id,) + ) + run_task = asyncio.create_task( + self._subtask_api.run_subtask_in_slot( + band_name, + slot_id, + subtask, + forward_successors=subtask_info.forward_successors, + finish_callback=finish_callback, + ) + ) + try: + return await asyncio.shield(run_task) + except asyncio.CancelledError as ex: + subtask_info.cancelling = True + cancel_coro = self._subtask_api.cancel_subtask_in_slot(band_name, slot_id) + try: + kill_timeout = ( + subtask_info.kill_timeout if self._enable_kill_slot else None + ) + await asyncio.wait_for(asyncio.shield(cancel_coro), kill_timeout) + except asyncio.TimeoutError: + logger.debug( + "Wait for subtask to cancel timed out (%s). " + "Start killing slot %d", + subtask_info.kill_timeout, + subtask_info.slot_id, + ) + await self._slot_manager_ref.kill_slot(band_name, slot_id) + except: # pragma: no cover + logger.exception("Unexpected errors raised when handling cancel") + raise + raise ex + except (OSError, MarsError) as ex: + if slot_id is not None: + # may encounter subprocess memory error + sub_pool_address = await self._slot_manager_ref.get_slot_address( + band_name, slot_id + ) + await mo.wait_actor_pool_recovered(sub_pool_address, self.address) + raise ex + finally: + logger.debug( + "Subtask %s running ended, slot_id=%r", subtask.subtask_id, slot_id + ) + + async def _execute_subtask_with_retry(self, subtask_info: SubtaskExecutionInfo): + subtask = subtask_info.subtask + try: + subtask_info.result = await self._call_with_retry( + self._execute_subtask_once, + subtask_info, + max_retries=subtask_info.max_retries if subtask.retryable else 0, + ) + except: # noqa: E722 # nosec # pylint: disable=bare-except + self._fill_result_with_exc(subtask_info) + finally: + self._subtask_executions.pop(subtask.subtask_id, None) + if not subtask_info.finish_future.done(): + subtask_info.finish_future.set_result(None) + + await self._report_subtask_results([subtask_info]) + await self._execution_queue_ref.release_slot( + subtask.subtask_id, errors="ignore" + ) + quota_ref = await self._get_band_quota_ref(subtask_info.band_name) + await quota_ref.release_quotas(tuple(subtask_info.quota_request.keys())) + return subtask_info.result + + @classmethod + async def _call_with_retry( + cls, + target_func: Callable, + subtask_info: SubtaskExecutionInfo, + max_retries: Optional[int] = None, + ): + subtask = subtask_info.subtask + max_retries = max_retries or subtask_info.max_retries + + def log_func(trial: int, exc_info: Tuple, retry: bool = True): + subtask_info.num_retries = trial + if retry: + logger.error( + "Rerun %s of subtask %s at attempt %d due to %s", + target_func, + subtask.subtask_id, + trial + 1, + exc_info[1], + ) + else: + logger.exception( + "Failed to rerun the %s of subtask %s, " + "num_retries: %s, max_retries: %s", + target_func, + subtask.subtask_id, + trial + 1, + max_retries, + exc_info=exc_info, + ) + + if subtask_info.max_retries <= 1: + return await target_func(subtask_info) + else: + return await call_with_retry( + functools.partial(target_func, subtask_info), + max_retries=max_retries, + error_callback=log_func, + ) + + @classmethod + def _fill_result_with_exc( + cls, + subtask_info: SubtaskExecutionInfo, + exc_info: Optional[Tuple] = None, + exc_cls: Type[Exception] = None, + ): + subtask = subtask_info.subtask + if exc_cls is not None: + try: + raise exc_cls + except: + exc_info = sys.exc_info() + exc_type, exc, tb = exc_info or sys.exc_info() + + if isinstance(exc, asyncio.CancelledError): + status = SubtaskStatus.cancelled + log_str = "Cancel" + else: + status = SubtaskStatus.errored + log_str = "Failed to" + + logger.exception( + "%s run subtask %s on band %s", + log_str, + subtask.subtask_id, + subtask_info.band_name, + exc_info=(exc_type, exc, tb), + ) + + subtask_info.result = SubtaskResult( + subtask_id=subtask.subtask_id, + session_id=subtask.session_id, + task_id=subtask.task_id, + progress=1.0, + status=status, + error=exc, + traceback=tb, + ) + + if not subtask_info.finish_future.done(): + subtask_info.finish_future.set_result(None) + + async def schedule_next(self, pred_subtask_id: str): + subtask_info = self._subtask_executions[pred_subtask_id] + succ_ids = set() + for result_chunk in subtask_info.subtask.chunk_graph.result_chunks: + try: + succ_ids.update(self._pred_key_mapping_dag.successors(result_chunk.key)) + except KeyError: + pass + + enqueue_tasks = [] + for succ_subtask_id in succ_ids: + try: + succ_subtask_info = self._subtask_caches[succ_subtask_id] + except KeyError: + continue + enqueue_task = asyncio.create_task( + self._forward_subtask_info(succ_subtask_info) + ) + enqueue_tasks.append(enqueue_task) + succ_subtask_info.aio_tasks.append(enqueue_task) + + if enqueue_tasks: + yield asyncio.wait(enqueue_tasks) + + await self._execution_queue_ref.release_slot(pred_subtask_id, errors="ignore") + + async def _forward_subtask_info(self, subtask_info: SubtaskExecutionInfo): + self._subtask_executions[subtask_info.subtask.subtask_id] = subtask_info + subtask_id = subtask_info.subtask.subtask_id + try: + await self._subtask_preparer.run(subtask_info, fail_fast=True) + await self._execution_queue_ref.put( + subtask_id, subtask_info.band_name, subtask_info.priority + ) + self.uncache_subtasks([subtask_id]) + except PrepareFastFailed: + self._subtask_executions.pop(subtask_id) diff --git a/mars/services/scheduling/worker/exec/core.py b/mars/services/scheduling/worker/exec/core.py new file mode 100644 index 0000000000..dcdcb891de --- /dev/null +++ b/mars/services/scheduling/worker/exec/core.py @@ -0,0 +1,69 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import asyncio +import logging +import sys +from dataclasses import dataclass, field +from typing import Callable, Dict, List, Optional, Tuple + +from .....core.base import MarsError +from .....utils import dataslots +from ....subtask import Subtask, SubtaskResult + +logger = logging.getLogger(__name__) + +# the default times to run subtask. +DEFAULT_SUBTASK_MAX_RETRIES = 0 + + +@dataslots +@dataclass +class SubtaskExecutionInfo: + subtask: Subtask + priority: Tuple + supervisor_address: str + band_name: str + aio_tasks: List[asyncio.Task] = field(default_factory=list) + result: SubtaskResult = field(default_factory=SubtaskResult) + cancelling: bool = False + max_retries: int = field(default=DEFAULT_SUBTASK_MAX_RETRIES) + num_retries: int = 0 + finish_future: asyncio.Future = field(default_factory=asyncio.Future) + quota_request: Optional[Dict] = None + slot_id: Optional[int] = None + kill_timeout: Optional[int] = None + # if True, when a subtask stops execution, its successors + # will be forwarded as soon as possible + forward_successors: bool = False + + +async def call_with_retry( + async_fun: Callable, max_retries: int, error_callback: Callable +): + max_retries = max_retries or 1 + + for trial in range(max_retries): + try: + return await async_fun() + except (OSError, MarsError): + if trial >= max_retries - 1: + error_callback(trial=trial, exc_info=sys.exc_info(), retry=False) + raise + error_callback(trial=trial, exc_info=sys.exc_info(), retry=True) + except asyncio.CancelledError: + raise + except: # noqa: E722 # nosec # pylint: disable=bare-except + error_callback(trial=trial, exc_info=sys.exc_info(), retry=False) + raise diff --git a/mars/services/scheduling/worker/exec/prepare.py b/mars/services/scheduling/worker/exec/prepare.py new file mode 100644 index 0000000000..436dd4a76f --- /dev/null +++ b/mars/services/scheduling/worker/exec/prepare.py @@ -0,0 +1,244 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import asyncio +import functools +import operator +from collections import defaultdict +from typing import Dict, Union + +from ..... import oscar as mo +from .....core.graph import DAG +from .....core.operand import Fetch, FetchShuffle +from .....lib.aio import alru_cache +from .....storage import StorageLevel +from .....utils import get_chunk_key_to_data_keys +from ....meta import MetaAPI +from ....storage import DataNotExist, StorageAPI +from ....subtask import Subtask +from ..quota import QuotaActor, QuotaInsufficientError +from .core import SubtaskExecutionInfo + + +class PrepareFastFailed(Exception): + pass + + +class SubtaskPreparer: + _storage_api: StorageAPI + + def __init__(self, address: str): + self._address = address + + @alru_cache(cache_exceptions=False) + async def _get_band_quota_ref(self, band: str) -> Union[mo.ActorRef, QuotaActor]: + return await mo.actor_ref(QuotaActor.gen_uid(band), address=self._address) + + async def _collect_input_sizes( + self, + subtask: Subtask, + supervisor_address: str, + band_name: str, + local_only: bool = False, + ): + graph = subtask.chunk_graph + key_to_sizes = dict() + + if local_only and any( + n.key for n in graph.iter_indep() if isinstance(n.op, FetchShuffle) + ): + raise DataNotExist + + fetch_keys = list( + set(n.key for n in graph.iter_indep() if isinstance(n.op, Fetch)) + ) + if not fetch_keys: + return key_to_sizes + + storage_api = await StorageAPI.create( + subtask.session_id, address=self._address, band_name=band_name + ) + data_infos = await storage_api.get_infos.batch( + *(storage_api.get_infos.delay(k, error="ignore") for k in fetch_keys) + ) + + # compute memory quota size. when data located in shared memory, the cost + # should be differences between deserialized memory cost and serialized cost, + # otherwise we should take deserialized memory cost + for key, infos in zip(fetch_keys, data_infos): + if not infos: + continue + level = functools.reduce(operator.or_, (info.level for info in infos)) + if level & StorageLevel.MEMORY: + mem_cost = max(0, infos[0].memory_size - infos[0].store_size) + else: + mem_cost = infos[0].memory_size + key_to_sizes[key] = (infos[0].store_size, mem_cost) + + non_local_keys = list(set(fetch_keys) - set(key_to_sizes.keys())) + if non_local_keys and local_only: + raise DataNotExist + + if non_local_keys: + meta_api = await MetaAPI.create( + subtask.session_id, address=supervisor_address + ) + fetch_metas = await meta_api.get_chunk_meta.batch( + *( + meta_api.get_chunk_meta.delay( + k, fields=["memory_size", "store_size"] + ) + for k in non_local_keys + ) + ) + for key, meta in zip(non_local_keys, fetch_metas): + key_to_sizes[key] = (meta["store_size"], meta["memory_size"]) + + return key_to_sizes + + @classmethod + def _estimate_sizes(cls, subtask: Subtask, input_sizes: Dict): + size_context = dict(input_sizes.items()) + graph = subtask.chunk_graph + + key_to_ops = defaultdict(set) + for n in graph: + key_to_ops[n.op.key].add(n.op) + key_to_ops = {k: list(v) for k, v in key_to_ops.items()} + + # condense op key graph + op_key_graph = DAG() + for n in graph.topological_iter(): + if n.op.key not in op_key_graph: + op_key_graph.add_node(n.op.key) + for succ in graph.iter_successors(n): + if succ.op.key not in op_key_graph: + op_key_graph.add_node(succ.op.key) + op_key_graph.add_edge(n.op.key, succ.op.key) + + key_stack = list(op_key_graph.iter_indep()) + pred_ref_count = {k: op_key_graph.count_predecessors(k) for k in op_key_graph} + succ_ref_count = {k: op_key_graph.count_successors(k) for k in op_key_graph} + + visited_op_keys = set() + total_memory_cost = 0 + max_memory_cost = sum(calc_size for _, calc_size in size_context.values()) + while key_stack: + key = key_stack.pop() + op = key_to_ops[key][0] + + if not isinstance(op, Fetch): + op.estimate_size(size_context, op) + + calc_cost = sum(size_context[out.key][1] for out in op.outputs) + total_memory_cost += calc_cost + max_memory_cost = max(total_memory_cost, max_memory_cost) + + if not isinstance(op, Fetch): + # when calculation result is stored, memory cost of calculation + # can be replaced with result memory cost + result_cost = sum(size_context[out.key][0] for out in op.outputs) + total_memory_cost += result_cost - calc_cost + + visited_op_keys.add(key) + + for succ_op_key in op_key_graph.iter_successors(key): + pred_ref_count[succ_op_key] -= 1 + if pred_ref_count[succ_op_key] == 0: + key_stack.append(succ_op_key) + + for pred_op_key in op_key_graph.iter_predecessors(key): + succ_ref_count[pred_op_key] -= 1 + if succ_ref_count[pred_op_key] == 0: + pred_op = key_to_ops[pred_op_key][0] + # when clearing fetches, subtract memory size, otherwise subtract store size + account_idx = 1 if isinstance(pred_op, Fetch) else 0 + pop_result_cost = sum( + size_context.pop(out.key, (0, 0))[account_idx] + for out in key_to_ops[pred_op_key][0].outputs + ) + total_memory_cost -= pop_result_cost + return sum(t[0] for t in size_context.values()), max_memory_cost + + async def _prepare_input_data(self, subtask: Subtask, band_name: str): + queries = [] + shuffle_queries = [] + storage_api = await StorageAPI.create( + subtask.session_id, address=self._address, band_name=band_name + ) + pure_dep_keys = set() + chunk_key_to_data_keys = get_chunk_key_to_data_keys(subtask.chunk_graph) + for n in subtask.chunk_graph: + pure_dep_keys.update( + inp.key + for inp, pure_dep in zip(n.inputs, n.op.pure_depends) + if pure_dep + ) + for chunk in subtask.chunk_graph: + if chunk.op.gpu: # pragma: no cover + to_fetch_band = band_name + else: + to_fetch_band = "numa-0" + if isinstance(chunk.op, Fetch): + queries.append( + storage_api.fetch.delay(chunk.key, band_name=to_fetch_band) + ) + elif isinstance(chunk.op, FetchShuffle): + for key in chunk_key_to_data_keys[chunk.key]: + shuffle_queries.append( + storage_api.fetch.delay( + key, band_name=to_fetch_band, error="ignore" + ) + ) + if queries: + await storage_api.fetch.batch(*queries) + if shuffle_queries: + # TODO(hks): The batch method doesn't accept different error arguments, + # combine them when it can. + await storage_api.fetch.batch(*shuffle_queries) + + async def run(self, subtask_info: SubtaskExecutionInfo, fail_fast: bool = False): + batch_quota_req = None + quota_ref = await self._get_band_quota_ref(subtask_info.band_name) + try: + subtask = subtask_info.subtask + try: + input_sizes = await self._collect_input_sizes( + subtask, + subtask_info.supervisor_address, + subtask_info.band_name, + local_only=fail_fast, + ) + except DataNotExist: + raise PrepareFastFailed from None + + _store_size, calc_size = await asyncio.to_thread( + self._estimate_sizes, subtask, input_sizes + ) + + try: + insufficient_quota = "raise" if fail_fast else "enqueue" + batch_quota_req = {(subtask.session_id, subtask.subtask_id): calc_size} + await quota_ref.request_batch_quota( + batch_quota_req, insufficient=insufficient_quota + ) + subtask_info.quota_request = batch_quota_req + except QuotaInsufficientError: + raise PrepareFastFailed from None + + await self._prepare_input_data(subtask_info.subtask, subtask_info.band_name) + except: # noqa: E722 # nosec # pylint: disable=bare-except + if batch_quota_req is not None: + await quota_ref.release_quotas(tuple(batch_quota_req.keys())) + raise diff --git a/mars/services/scheduling/worker/exec/tests/__init__.py b/mars/services/scheduling/worker/exec/tests/__init__.py new file mode 100644 index 0000000000..c71e83c08e --- /dev/null +++ b/mars/services/scheduling/worker/exec/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/mars/services/scheduling/worker/exec/tests/test_exec.py b/mars/services/scheduling/worker/exec/tests/test_exec.py new file mode 100644 index 0000000000..f2e0b2e72d --- /dev/null +++ b/mars/services/scheduling/worker/exec/tests/test_exec.py @@ -0,0 +1,546 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import asyncio +import os +import tempfile +import time +import uuid +from contextlib import asynccontextmanager + +import numpy as np +import pytest + +from ...... import oscar as mo +from ...... import remote as mr +from ......core import ( + ChunkGraph, + TileableGraph, + TileableGraphBuilder, + ChunkGraphBuilder, +) +from ......core.base import MarsError +from ......remote.core import RemoteFunction +from ......tensor.fetch import TensorFetch +from ......tensor.arithmetic import TensorTreeAdd +from ......utils import Timer, build_fetch_chunk +from .....cluster import MockClusterAPI +from .....lifecycle import MockLifecycleAPI +from .....meta import MockMetaAPI +from .....mutable import MockMutableAPI +from .....session import MockSessionAPI +from .....storage import MockStorageAPI +from .....subtask import MockSubtaskAPI, Subtask, SubtaskStatus +from .....task.supervisor.manager import TaskManagerActor +from ...quota import QuotaActor +from ...queues import SubtaskPrepareQueueActor, SubtaskExecutionQueueActor +from ...slotmanager import SlotManagerActor +from ..actor import SubtaskExecutionActor + + +class CancelDetectActorMixin: + @asynccontextmanager + async def _delay_method(self): + delay_fetch_event = getattr(self, "_delay_fetch_event", None) + delay_wait_event = getattr(self, "_delay_wait_event", None) + try: + if delay_fetch_event is not None: + delay_fetch_event.set() + if delay_wait_event is not None: + await delay_wait_event.wait() + yield + except asyncio.CancelledError: + self._is_cancelled = True + raise + + def set_delay_fetch_event( + self, fetch_event: asyncio.Event, wait_event: asyncio.Event + ): + setattr(self, "_delay_fetch_event", fetch_event) + setattr(self, "_delay_wait_event", wait_event) + + def get_is_cancelled(self): + return getattr(self, "_is_cancelled", False) + + +class MockQuotaActor(QuotaActor, CancelDetectActorMixin): + def __init__(self, *args, **kw): + super().__init__(*args, **kw) + self._batch_quota_reqs = {} + + async def request_batch_quota(self, batch, insufficient: str = "enqueue"): + async with self._delay_method(): + res = super().request_batch_quota(batch, insufficient=insufficient) + self._batch_quota_reqs.update(batch) + return res + + def get_batch_quota_reqs(self, quota_key): + return self._batch_quota_reqs[quota_key] + + +class MockSubtaskPrepareQueueActor(SubtaskPrepareQueueActor, CancelDetectActorMixin): + async def get(self, band_name: str): + val = await super().get(band_name) + async with self._delay_method(): + return val + + +class MockSubtaskExecutionQueueActor( + SubtaskExecutionQueueActor, CancelDetectActorMixin +): + async def get(self, band_name: str): + val = await super().get(band_name) + async with self._delay_method(): + return val + + +class MockTaskManager(mo.Actor): + def __init__(self): + self._results = [] + + def set_subtask_result(self, result): + self._results.append(result) + + def get_results(self): + return self._results + + +@pytest.fixture +async def actor_pool(request): + n_slots, enable_kill, max_retries = request.param + pool = await mo.create_actor_pool( + "127.0.0.1", labels=[None] + ["numa-0"] * n_slots, n_process=n_slots + ) + + async with pool: + session_id = "test_session" + await MockClusterAPI.create( + pool.external_address, band_to_slots={"numa-0": n_slots} + ) + await MockSessionAPI.create(pool.external_address, session_id=session_id) + meta_api = await MockMetaAPI.create(session_id, pool.external_address) + await MockLifecycleAPI.create(session_id, pool.external_address) + await MockSubtaskAPI.create(pool.external_address) + await MockMutableAPI.create(session_id, pool.external_address) + storage_api = await MockStorageAPI.create( + session_id, + pool.external_address, + ) + + slot_manager_ref = await mo.create_actor( + SlotManagerActor, + uid=SlotManagerActor.default_uid(), + address=pool.external_address, + ) + + prepare_queue_ref = await mo.create_actor( + MockSubtaskPrepareQueueActor, + uid=SubtaskPrepareQueueActor.default_uid(), + address=pool.external_address, + ) + exec_queue_ref = await mo.create_actor( + MockSubtaskExecutionQueueActor, + uid=SubtaskExecutionQueueActor.default_uid(), + address=pool.external_address, + ) + # create quota actor + quota_ref = await mo.create_actor( + MockQuotaActor, + "numa-0", + 102400, + uid=QuotaActor.gen_uid("numa-0"), + address=pool.external_address, + ) + + # create mock task manager actor + task_manager_ref = await mo.create_actor( + MockTaskManager, + uid=TaskManagerActor.gen_uid(session_id), + address=pool.external_address, + ) + + # create assigner actor + execution_ref = await mo.create_actor( + SubtaskExecutionActor, + subtask_max_retries=max_retries, + enable_kill_slot=enable_kill, + uid=SubtaskExecutionActor.default_uid(), + address=pool.external_address, + ) + + try: + yield pool, session_id, meta_api, storage_api, execution_ref + finally: + await mo.destroy_actor(execution_ref) + await mo.destroy_actor(prepare_queue_ref) + await mo.destroy_actor(exec_queue_ref) + await mo.destroy_actor(quota_ref) + await mo.destroy_actor(task_manager_ref) + await mo.destroy_actor(slot_manager_ref) + await MockStorageAPI.cleanup(pool.external_address) + await MockSubtaskAPI.cleanup(pool.external_address) + await MockClusterAPI.cleanup(pool.external_address) + await MockMutableAPI.cleanup(session_id, pool.external_address) + + +@pytest.mark.asyncio +@pytest.mark.parametrize("actor_pool", [(1, True, 0)], indirect=True) +async def test_execute_tensor(actor_pool): + pool, session_id, meta_api, storage_api, execution_ref = actor_pool + + async def build_test_subtask(subtask_name): + data1 = np.random.rand(10, 10) + data2 = np.random.rand(10, 10) + + input1_key = f"{subtask_name}_input1" + input1 = TensorFetch( + key=input1_key, source_key=input1_key, dtype=np.dtype(int) + ).new_chunk([]) + + input2_key = f"{subtask_name}_input2" + input2 = TensorFetch( + key=input2_key, source_key=input2_key, dtype=np.dtype(int) + ).new_chunk([]) + + result_chunk = TensorTreeAdd(args=[input1, input2]).new_chunk( + [input1, input2], shape=data1.shape, dtype=data1.dtype + ) + + await meta_api.set_chunk_meta( + input1, + memory_size=data1.nbytes, + store_size=data1.nbytes, + bands=[(pool.external_address, "numa-0")], + ) + await meta_api.set_chunk_meta( + input2, + memory_size=data1.nbytes, + store_size=data2.nbytes, + bands=[(pool.external_address, "numa-0")], + ) + # todo use different storage level when storage ready + await storage_api.put(input1.key, data1) + await storage_api.put(input2.key, data2) + + chunk_graph = ChunkGraph([result_chunk]) + chunk_graph.add_node(input1) + chunk_graph.add_node(input2) + chunk_graph.add_node(result_chunk) + chunk_graph.add_edge(input1, result_chunk) + chunk_graph.add_edge(input2, result_chunk) + + return ( + Subtask(subtask_name, session_id=session_id, chunk_graph=chunk_graph), + (data1, data2), + ) + + subtask1, data_group1 = await build_test_subtask("subtask1") + subtask2, data_group2 = await build_test_subtask("subtask2") + + await execution_ref.submit_subtasks( + [subtask1, subtask2], [(0,), (0,)], pool.external_address, "numa-0" + ) + await execution_ref.wait_subtasks([subtask1.subtask_id, subtask2.subtask_id]) + + for subtask, (data1, data2) in [(subtask1, data_group1), (subtask2, data_group2)]: + result_chunk = subtask.chunk_graph.result_chunks[0] + # check if results are correct + result = await storage_api.get(result_chunk.key) + np.testing.assert_array_equal(data1 + data2, result) + + # check if quota computations are correct + quota_ref = await mo.actor_ref( + QuotaActor.gen_uid("numa-0"), address=pool.external_address + ) + quota = await quota_ref.get_batch_quota_reqs( + (subtask.session_id, subtask.subtask_id) + ) + assert quota == data1.nbytes + + # check if metas are correct + result_meta = await meta_api.get_chunk_meta(result_chunk.key) + assert result_meta["object_id"] == result_chunk.key + assert result_meta["shape"] == result.shape + + +_cancel_phases = [ + "immediately", + "prepare-queue", + "prepare", + "execute-queue", + "execute", +] + + +@pytest.mark.asyncio +@pytest.mark.parametrize( + "actor_pool,cancel_phase", + [((1, True, 0), phase) for phase in _cancel_phases], + indirect=["actor_pool"], +) +async def test_execute_with_cancel(actor_pool, cancel_phase): + pool, session_id, meta_api, storage_api, execution_ref = actor_pool + delay_fetch_event = asyncio.Event() + delay_wait_event = asyncio.Event() + + # config for different phases + ref_to_delay = None + if cancel_phase == "prepare-queue": + ref_to_delay = await mo.actor_ref( + SubtaskPrepareQueueActor.default_uid(), + address=pool.external_address, + ) + elif cancel_phase == "prepare": + ref_to_delay = await mo.actor_ref( + QuotaActor.gen_uid("numa-0"), address=pool.external_address + ) + elif cancel_phase == "execute-queue": + ref_to_delay = await mo.actor_ref( + SubtaskExecutionQueueActor.default_uid(), + address=pool.external_address, + ) + if ref_to_delay is not None: + await ref_to_delay.set_delay_fetch_event(delay_fetch_event, delay_wait_event) + else: + delay_fetch_event.set() + + def delay_fun(delay, _inp1): + if not ref_to_delay: + time.sleep(delay) + return (delay,) + + input1 = TensorFetch( + key="input1", source_key="input1", dtype=np.dtype(int) + ).new_chunk([]) + remote_result = RemoteFunction( + function=delay_fun, function_args=[100, input1], function_kwargs={}, n_output=1 + ).new_chunk([input1]) + + data1 = np.random.rand(10, 10) + await meta_api.set_chunk_meta( + input1, + memory_size=data1.nbytes, + store_size=data1.nbytes, + bands=[(pool.external_address, "numa-0")], + ) + await storage_api.put(input1.key, data1) + + chunk_graph = ChunkGraph([remote_result]) + chunk_graph.add_node(input1) + chunk_graph.add_node(remote_result) + chunk_graph.add_edge(input1, remote_result) + + subtask = Subtask( + f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph + ) + await execution_ref.submit_subtasks( + [subtask], [(0,)], pool.external_address, "numa-0" + ) + aiotask = asyncio.create_task(execution_ref.wait_subtasks([subtask.subtask_id])) + if ref_to_delay is not None: + await delay_fetch_event.wait() + else: + if cancel_phase != "immediately": + await asyncio.sleep(1) + + with Timer() as timer: + await asyncio.wait_for( + execution_ref.cancel_subtasks([subtask.subtask_id], kill_timeout=1), + timeout=30, + ) + [r] = await asyncio.wait_for(aiotask, timeout=30) + assert r.status == SubtaskStatus.cancelled + assert timer.duration < 15 + + # check for different phases + if ref_to_delay is not None: + if not cancel_phase.endswith("queue"): + assert await ref_to_delay.get_is_cancelled() + delay_wait_event.set() + + # test if slot is restored + remote_tileable = mr.spawn(delay_fun, args=(0.5, None)) + graph = TileableGraph([remote_tileable.data]) + next(TileableGraphBuilder(graph).build()) + + chunk_graph = next(ChunkGraphBuilder(graph, fuse_enabled=False).build()) + + subtask = Subtask( + f"test_subtask2_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph + ) + await execution_ref.submit_subtasks( + [subtask], [(0,)], pool.external_address, "numa-0" + ) + await asyncio.wait_for(execution_ref.wait_subtasks([subtask]), timeout=30) + + +@pytest.mark.asyncio +@pytest.mark.parametrize("actor_pool", [(1, False, 0)], indirect=True) +async def test_cancel_without_kill(actor_pool): + pool, session_id, meta_api, storage_api, execution_ref = actor_pool + executed_file = os.path.join( + tempfile.gettempdir(), f"mars_test_cancel_without_kill_{os.getpid()}.tmp" + ) + + def delay_fun(delay): + import mars + + open(executed_file, "w").close() + time.sleep(delay) + mars._slot_marker = 1 + return delay + + def check_fun(): + import mars + + return getattr(mars, "_slot_marker", False) + + remote_result = RemoteFunction( + function=delay_fun, function_args=[2], function_kwargs={} + ).new_chunk([]) + chunk_graph = ChunkGraph([remote_result]) + chunk_graph.add_node(remote_result) + + subtask = Subtask( + f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph + ) + await execution_ref.submit_subtasks( + [subtask], [(0,)], pool.external_address, "numa-0" + ) + aiotask = asyncio.create_task(execution_ref.wait_subtasks([subtask.subtask_id])) + await asyncio.sleep(0.5) + + await asyncio.wait_for( + execution_ref.cancel_subtasks([subtask.subtask_id], kill_timeout=1), + timeout=30, + ) + [r] = await asyncio.wait_for(aiotask, timeout=30) + assert r.status == SubtaskStatus.cancelled + + remote_result = RemoteFunction( + function=check_fun, function_args=[], function_kwargs={} + ).new_chunk([]) + chunk_graph = ChunkGraph([remote_result]) + chunk_graph.add_node(remote_result) + + subtask = Subtask( + f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph + ) + await execution_ref.submit_subtasks( + [subtask], [(0,)], pool.external_address, "numa-0" + ) + await execution_ref.wait_subtasks([subtask.subtask_id]) + + # check if slots not killed (or slot assignment may be cancelled) + if os.path.exists(executed_file): + assert await storage_api.get(remote_result.key) + os.unlink(executed_file) + + +@pytest.mark.asyncio +@pytest.mark.parametrize("actor_pool", [(1, False, 2)], indirect=True) +async def test_retry_execution(actor_pool): + with tempfile.TemporaryDirectory(prefix="mars_test_retry_exec_") as tempdir: + pool, session_id, meta_api, storage_api, execution_ref = actor_pool + + def error_fun(count_file, max_fail): + file_path = os.path.join(tempdir, count_file) + try: + cnt = int(open(file_path, "r").read()) + except OSError: + cnt = 0 + + if cnt < max_fail: + with open(file_path, "w") as f: + f.write(str(cnt + 1)) + raise MarsError(f"Error No {cnt}") + + remote_result = RemoteFunction( + function=error_fun, function_args=["subtask1", 1], function_kwargs={} + ).new_chunk([]) + chunk_graph = ChunkGraph([remote_result]) + chunk_graph.add_node(remote_result) + + subtask = Subtask( + f"test_subtask_{uuid.uuid4()}", + session_id=session_id, + chunk_graph=chunk_graph, + ) + await execution_ref.submit_subtasks( + [subtask], [(0,)], pool.external_address, "numa-0" + ) + [r] = await execution_ref.wait_subtasks([subtask.subtask_id]) + assert r.status == SubtaskStatus.succeeded + assert int(open(os.path.join(tempdir, "subtask1"), "r").read()) == 1 + + remote_result = RemoteFunction( + function=error_fun, function_args=["subtask2", 2], function_kwargs={} + ).new_chunk([]) + chunk_graph = ChunkGraph([remote_result]) + chunk_graph.add_node(remote_result) + + subtask = Subtask( + f"test_subtask_{uuid.uuid4()}", + session_id=session_id, + chunk_graph=chunk_graph, + ) + await execution_ref.submit_subtasks( + [subtask], [(0,)], pool.external_address, "numa-0" + ) + [r] = await execution_ref.wait_subtasks([subtask.subtask_id]) + assert r.status == SubtaskStatus.errored + assert int(open(os.path.join(tempdir, "subtask2"), "r").read()) == 2 + + +@pytest.mark.asyncio +@pytest.mark.parametrize("actor_pool", [(1, False, 0)], indirect=True) +async def test_cached_successors(actor_pool): + pool, session_id, meta_api, storage_api, execution_ref = actor_pool + + data1 = np.random.rand(5, 5) + data2 = np.random.rand(5, 5) + + def func1(): + return (data1,) + + def func2(v): + return (v + data2,) + + result1 = RemoteFunction(function=func1, n_output=1).new_chunk([]) + chunk_graph1 = ChunkGraph([result1]) + chunk_graph1.add_node(result1) + subtask1 = Subtask( + f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph1 + ) + + fetch1 = build_fetch_chunk(result1) + result2 = RemoteFunction( + function=func2, function_args=[fetch1], n_output=1 + ).new_chunk([fetch1]) + chunk_graph2 = ChunkGraph([result2]) + chunk_graph2.add_node(fetch1) + chunk_graph2.add_node(result2) + chunk_graph2.add_edge(fetch1, result2) + subtask2 = Subtask( + f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph2 + ) + + await execution_ref.cache_subtasks( + [subtask2], [(0,)], pool.external_address, "numa-0" + ) + await execution_ref.submit_subtasks( + [subtask1], [(0,)], pool.external_address, "numa-0" + ) + [r] = await execution_ref.wait_subtasks([subtask1.subtask_id]) + [r] = await execution_ref.wait_subtasks([subtask2.subtask_id]) diff --git a/mars/services/scheduling/worker/exec/tests/test_prepare.py b/mars/services/scheduling/worker/exec/tests/test_prepare.py new file mode 100644 index 0000000000..c71e83c08e --- /dev/null +++ b/mars/services/scheduling/worker/exec/tests/test_prepare.py @@ -0,0 +1,13 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/mars/services/scheduling/worker/execution.py b/mars/services/scheduling/worker/execution.py deleted file mode 100644 index bf3d5860ac..0000000000 --- a/mars/services/scheduling/worker/execution.py +++ /dev/null @@ -1,478 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import asyncio -import functools -import logging -import operator -import pprint -import sys -from collections import defaultdict -from dataclasses import dataclass, field -from typing import Dict, Optional, Union - -from .... import oscar as mo -from ....core.graph import DAG -from ....core.operand import Fetch, FetchShuffle -from ....lib.aio import alru_cache -from ....oscar.errors import MarsError -from ....storage import StorageLevel -from ....utils import dataslots, get_chunk_key_to_data_keys -from ...cluster import ClusterAPI -from ...meta import MetaAPI -from ...storage import StorageAPI -from ...subtask import Subtask, SubtaskAPI, SubtaskResult, SubtaskStatus -from .workerslot import BandSlotManagerActor -from .quota import QuotaActor - -logger = logging.getLogger(__name__) - -# the default times to run subtask. -DEFAULT_SUBTASK_MAX_RETRIES = 0 - - -@dataslots -@dataclass -class SubtaskExecutionInfo: - aio_task: asyncio.Task - band_name: str - supervisor_address: str - result: SubtaskResult = field(default_factory=SubtaskResult) - cancelling: bool = False - max_retries: int = 0 - num_retries: int = 0 - slot_id: Optional[int] = None - kill_timeout: Optional[int] = None - - -async def _retry_run( - subtask: Subtask, subtask_info: SubtaskExecutionInfo, target_async_func, *args -): - assert subtask_info.num_retries >= 0 - assert subtask_info.max_retries >= 0 - - while True: - try: - return await target_async_func(*args) - except (OSError, MarsError) as ex: - if subtask_info.num_retries < subtask_info.max_retries: - logger.error( - "Rerun[%s/%s] the %s of subtask %s due to %s.", - subtask_info.num_retries, - subtask_info.max_retries, - target_async_func, - subtask.subtask_id, - ex, - ) - subtask_info.num_retries += 1 - continue - if subtask_info.max_retries > 0: - message = ( - f"Exceed max rerun[{subtask_info.num_retries}/{subtask_info.max_retries}]:" - f" {target_async_func} of subtask {subtask.subtask_id} due to {ex}." - ) - logger.error(message) - - class _ExceedMaxRerun(type(ex)): - pass - - raise _ExceedMaxRerun(message).with_traceback(ex.__traceback__) - else: - raise ex - except asyncio.CancelledError: - raise - except Exception as ex: - if subtask_info.max_retries > 0: - message = ( - f"Failed to rerun the {target_async_func} of subtask {subtask.subtask_id}, " - f"num_retries: {subtask_info.num_retries}, max_retries: {subtask_info.max_retries} " - f"due to unhandled exception: {ex}." - ) - logger.error(message) - - class _UnhandledException(type(ex)): - pass - - raise _UnhandledException(message).with_traceback(ex.__traceback__) - else: - raise ex - - -def _fill_subtask_result_with_exception( - subtask: Subtask, subtask_info: SubtaskExecutionInfo -): - _, exc, tb = sys.exc_info() - if isinstance(exc, asyncio.CancelledError): - status = SubtaskStatus.cancelled - log_str = "Cancel" - else: - status = SubtaskStatus.errored - log_str = "Failed to" - logger.exception( - "%s run subtask %s on band %s", - log_str, - subtask.subtask_id, - subtask_info.band_name, - ) - subtask_info.result.status = status - subtask_info.result.progress = 1.0 - subtask_info.result.error = exc - subtask_info.result.traceback = tb - - -class SubtaskExecutionActor(mo.StatelessActor): - _subtask_info: Dict[str, SubtaskExecutionInfo] - - def __init__( - self, - subtask_max_retries: int = DEFAULT_SUBTASK_MAX_RETRIES, - enable_kill_slot: bool = True, - ): - self._cluster_api = None - self._global_slot_ref = None - self._subtask_max_retries = subtask_max_retries - self._enable_kill_slot = enable_kill_slot - - self._subtask_info = dict() - - async def __post_create__(self): - self._cluster_api = await ClusterAPI.create(self.address) - - @alru_cache(cache_exceptions=False) - async def _get_slot_manager_ref( - self, band: str - ) -> Union[mo.ActorRef, BandSlotManagerActor]: - return await mo.actor_ref( - BandSlotManagerActor.gen_uid(band), address=self.address - ) - - @alru_cache(cache_exceptions=False) - async def _get_band_quota_ref(self, band: str) -> Union[mo.ActorRef, QuotaActor]: - return await mo.actor_ref(QuotaActor.gen_uid(band), address=self.address) - - async def _prepare_input_data(self, subtask: Subtask, band_name: str): - queries = [] - shuffle_queries = [] - storage_api = await StorageAPI.create( - subtask.session_id, address=self.address, band_name=band_name - ) - pure_dep_keys = set() - chunk_key_to_data_keys = get_chunk_key_to_data_keys(subtask.chunk_graph) - for n in subtask.chunk_graph: - pure_dep_keys.update( - inp.key - for inp, pure_dep in zip(n.inputs, n.op.pure_depends) - if pure_dep - ) - for chunk in subtask.chunk_graph: - if chunk.op.gpu: # pragma: no cover - to_fetch_band = band_name - else: - to_fetch_band = "numa-0" - if isinstance(chunk.op, Fetch): - queries.append( - storage_api.fetch.delay(chunk.key, band_name=to_fetch_band) - ) - elif isinstance(chunk.op, FetchShuffle): - for key in chunk_key_to_data_keys[chunk.key]: - shuffle_queries.append( - storage_api.fetch.delay( - key, band_name=to_fetch_band, error="ignore" - ) - ) - if queries: - await storage_api.fetch.batch(*queries) - if shuffle_queries: - # TODO(hks): The batch method doesn't accept different error arguments, - # combine them when it can. - await storage_api.fetch.batch(*shuffle_queries) - - async def _collect_input_sizes( - self, subtask: Subtask, supervisor_address: str, band_name: str - ): - graph = subtask.chunk_graph - sizes = dict() - - fetch_keys = list( - set(n.key for n in graph.iter_indep() if isinstance(n.op, Fetch)) - ) - if not fetch_keys: - return sizes - - storage_api = await StorageAPI.create( - subtask.session_id, address=self.address, band_name=band_name - ) - meta_api = await MetaAPI.create(subtask.session_id, address=supervisor_address) - - fetch_metas = await meta_api.get_chunk_meta.batch( - *( - meta_api.get_chunk_meta.delay(k, fields=["memory_size", "store_size"]) - for k in fetch_keys - ) - ) - data_infos = await storage_api.get_infos.batch( - *(storage_api.get_infos.delay(k) for k in fetch_keys) - ) - - # compute memory quota size. when data located in shared memory, the cost - # should be differences between deserialized memory cost and serialized cost, - # otherwise we should take deserialized memory cost - for key, meta, infos in zip(fetch_keys, fetch_metas, data_infos): - level = functools.reduce(operator.or_, (info.level for info in infos)) - if level & StorageLevel.MEMORY: - mem_cost = max(0, meta["memory_size"] - meta["store_size"]) - else: - mem_cost = meta["memory_size"] - sizes[key] = (meta["store_size"], mem_cost) - - return sizes - - @classmethod - def _estimate_sizes(cls, subtask: Subtask, input_sizes: Dict): - size_context = dict(input_sizes.items()) - graph = subtask.chunk_graph - - key_to_ops = defaultdict(set) - for n in graph: - key_to_ops[n.op.key].add(n.op) - key_to_ops = {k: list(v) for k, v in key_to_ops.items()} - - # condense op key graph - op_key_graph = DAG() - for n in graph.topological_iter(): - if n.op.key not in op_key_graph: - op_key_graph.add_node(n.op.key) - for succ in graph.iter_successors(n): - if succ.op.key not in op_key_graph: - op_key_graph.add_node(succ.op.key) - op_key_graph.add_edge(n.op.key, succ.op.key) - - key_stack = list(op_key_graph.iter_indep()) - pred_ref_count = {k: op_key_graph.count_predecessors(k) for k in op_key_graph} - succ_ref_count = {k: op_key_graph.count_successors(k) for k in op_key_graph} - - visited_op_keys = set() - total_memory_cost = 0 - max_memory_cost = sum(calc_size for _, calc_size in size_context.values()) - while key_stack: - key = key_stack.pop() - op = key_to_ops[key][0] - - if not isinstance(op, Fetch): - op.estimate_size(size_context, op) - - calc_cost = sum(size_context[out.key][1] for out in op.outputs) - total_memory_cost += calc_cost - max_memory_cost = max(total_memory_cost, max_memory_cost) - - if not isinstance(op, Fetch): - # when calculation result is stored, memory cost of calculation - # can be replaced with result memory cost - result_cost = sum(size_context[out.key][0] for out in op.outputs) - total_memory_cost += result_cost - calc_cost - - visited_op_keys.add(key) - - for succ_op_key in op_key_graph.iter_successors(key): - pred_ref_count[succ_op_key] -= 1 - if pred_ref_count[succ_op_key] == 0: - key_stack.append(succ_op_key) - - for pred_op_key in op_key_graph.iter_predecessors(key): - succ_ref_count[pred_op_key] -= 1 - if succ_ref_count[pred_op_key] == 0: - pred_op = key_to_ops[pred_op_key][0] - # when clearing fetches, subtract memory size, otherwise subtract store size - account_idx = 1 if isinstance(pred_op, Fetch) else 0 - pop_result_cost = sum( - size_context.pop(out.key, (0, 0))[account_idx] - for out in key_to_ops[pred_op_key][0].outputs - ) - total_memory_cost -= pop_result_cost - return sum(t[0] for t in size_context.values()), max_memory_cost - - @classmethod - def _check_cancelling(cls, subtask_info: SubtaskExecutionInfo): - if subtask_info.cancelling: - raise asyncio.CancelledError - - async def internal_run_subtask(self, subtask: Subtask, band_name: str): - subtask_api = SubtaskAPI(self.address) - subtask_info = self._subtask_info[subtask.subtask_id] - subtask_info.result = SubtaskResult( - subtask_id=subtask.subtask_id, - session_id=subtask.session_id, - task_id=subtask.task_id, - status=SubtaskStatus.pending, - ) - try: - await _retry_run( - subtask, subtask_info, self._prepare_input_data, subtask, band_name - ) - - input_sizes = await self._collect_input_sizes( - subtask, subtask_info.supervisor_address, band_name - ) - _store_size, calc_size = await asyncio.to_thread( - self._estimate_sizes, subtask, input_sizes - ) - self._check_cancelling(subtask_info) - - batch_quota_req = {(subtask.session_id, subtask.subtask_id): calc_size} - subtask_info.result = await self._retry_run_subtask( - subtask, band_name, subtask_api, batch_quota_req - ) - except: # noqa: E722 # pylint: disable=bare-except - _fill_subtask_result_with_exception(subtask, subtask_info) - finally: - # make sure new slot usages are uploaded in time - try: - slot_manager_ref = await self._get_slot_manager_ref(band_name) - await slot_manager_ref.upload_slot_usages(periodical=False) - except: # noqa: E722 # pylint: disable=bare-except - _fill_subtask_result_with_exception(subtask, subtask_info) - finally: - # pop the subtask info at the end is to cancel the job. - self._subtask_info.pop(subtask.subtask_id, None) - return subtask_info.result - - async def _retry_run_subtask( - self, subtask: Subtask, band_name: str, subtask_api: SubtaskAPI, batch_quota_req - ): - quota_ref = await self._get_band_quota_ref(band_name) - slot_manager_ref = await self._get_slot_manager_ref(band_name) - subtask_info = self._subtask_info[subtask.subtask_id] - assert subtask_info.num_retries >= 0 - assert subtask_info.max_retries >= 0 - - async def _run_subtask_once(): - aiotask = None - slot_id = None - try: - await quota_ref.request_batch_quota(batch_quota_req) - self._check_cancelling(subtask_info) - - slot_id = await slot_manager_ref.acquire_free_slot( - (subtask.session_id, subtask.subtask_id) - ) - self._check_cancelling(subtask_info) - - subtask_info.result.status = SubtaskStatus.running - aiotask = asyncio.create_task( - subtask_api.run_subtask_in_slot(band_name, slot_id, subtask) - ) - return await asyncio.shield(aiotask) - except asyncio.CancelledError as ex: - # make sure allocated slots are traced - if slot_id is None: # pragma: no cover - slot_id = await slot_manager_ref.get_subtask_slot( - (subtask.session_id, subtask.subtask_id) - ) - try: - if aiotask is not None: - await asyncio.wait_for( - asyncio.shield( - subtask_api.cancel_subtask_in_slot(band_name, slot_id) - ), - subtask_info.kill_timeout, - ) - except asyncio.TimeoutError: - logger.debug( - "Wait for subtask to cancel timed out (%s). " - "Start killing slot %d", - subtask_info.kill_timeout, - slot_id, - ) - await slot_manager_ref.kill_slot(slot_id) - sub_pool_address = await slot_manager_ref.get_slot_address(slot_id) - await mo.wait_actor_pool_recovered(sub_pool_address, self.address) - except: # pragma: no cover - logger.exception("Unexpected errors raised when handling cancel") - raise - finally: - raise ex - except (OSError, MarsError) as ex: - if slot_id is not None: - # may encounter subprocess memory error - sub_pool_address = await slot_manager_ref.get_slot_address(slot_id) - await mo.wait_actor_pool_recovered(sub_pool_address, self.address) - raise ex - finally: - logger.debug("Subtask running ended, slot_id=%r", slot_id) - if slot_id is not None: - await slot_manager_ref.release_free_slot( - slot_id, (subtask.session_id, subtask.subtask_id) - ) - await quota_ref.release_quotas(tuple(batch_quota_req.keys())) - - # TODO(fyrestone): For the retryable op, we should rerun it when - # any exceptions occurred. - if subtask.retryable: - return await _retry_run(subtask, subtask_info, _run_subtask_once) - else: - try: - return await _run_subtask_once() - except Exception as e: - unretryable_op = [ - chunk.op - for chunk in subtask.chunk_graph - if not getattr(chunk.op, "retryable", True) - ] - message = ( - f"Run subtask failed due to {e}, the subtask {subtask.subtask_id} is " - f"not retryable, it contains unretryable op: \n" - f"{pprint.pformat(unretryable_op)}" - ) - logger.error(message) - - class _UnretryableException(type(e)): - pass - - raise _UnretryableException(message).with_traceback(e.__traceback__) - - async def run_subtask( - self, subtask: Subtask, band_name: str, supervisor_address: str - ): - with mo.debug.no_message_trace(): - task = asyncio.create_task( - self.ref().internal_run_subtask(subtask, band_name) - ) - - # the extra_config may be None. the extra config overwrites the default value. - subtask_max_retries = ( - subtask.extra_config.get("subtask_max_retries") - if subtask.extra_config - else None - ) - if subtask_max_retries is None: - subtask_max_retries = self._subtask_max_retries - - self._subtask_info[subtask.subtask_id] = SubtaskExecutionInfo( - task, band_name, supervisor_address, max_retries=subtask_max_retries - ) - return await task - - async def cancel_subtask(self, subtask_id: str, kill_timeout: Optional[int] = 5): - try: - subtask_info = self._subtask_info[subtask_id] - except KeyError: - return - - kill_timeout = kill_timeout if self._enable_kill_slot else None - if not subtask_info.cancelling: - subtask_info.kill_timeout = kill_timeout - subtask_info.cancelling = True - subtask_info.aio_task.cancel() - - await subtask_info.aio_task diff --git a/mars/services/scheduling/worker/queues.py b/mars/services/scheduling/worker/queues.py new file mode 100644 index 0000000000..7cf5fb0e07 --- /dev/null +++ b/mars/services/scheduling/worker/queues.py @@ -0,0 +1,184 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import asyncio +import logging +from typing import Dict, NamedTuple, Tuple + +from .... import oscar as mo +from ...cluster import ClusterAPI + +logger = logging.getLogger(__name__) + + +class QueueItem(NamedTuple): + subtask_id: str + priority: tuple + + def __lt__(self, other: "QueueItem"): + return self.priority > other.priority + + +class SlotsContainer(list): + def __init__(self, *args, **kwargs): + super().__init__(*args, **kwargs) + self._item_set = set(self) + + def append(self, item) -> None: + super().append(item) + self._item_set.add(item) + + def insert(self, index, item) -> None: + super().insert(index, item) + self._item_set.add(index) + + def pop(self, index=None): + if index is not None: + popped = super().pop(index) + else: + popped = super().pop() + self._item_set.remove(popped) + return popped + + def __contains__(self, item): + return item in self._item_set + + +class SubtaskPriorityQueueActor(mo.StatelessActor): + _subtask_id_to_item: Dict[str, QueueItem] + _subtask_id_to_band_slot: Dict[str, Tuple[str, str]] + + _band_n_slots: Dict[str, int] + _band_active_slots: Dict[str, SlotsContainer] + _semaphores: Dict[str, asyncio.Semaphore] + _queues: Dict[str, asyncio.PriorityQueue] + + def __init__(self): + self._subtask_id_to_item = dict() + self._subtask_id_to_band_slot = dict() + + self._semaphores = dict() + self._queues = dict() + self._band_n_slots = dict() + self._band_active_slots = dict() + + async def __post_create__(self): + cluster_api = await ClusterAPI.create(self.address) + bands = await cluster_api.get_bands() + + for (_a, band_name), slot_num in bands.items(): + self._init_band(band_name, slot_num) + + def _init_band(self, band_name: str, slot_num: int): + self._band_n_slots[band_name] = slot_num + self._band_active_slots[band_name] = SlotsContainer(range(slot_num)) + self._semaphores[band_name] = asyncio.Semaphore(slot_num) + self._queues[band_name] = asyncio.PriorityQueue() + + @mo.extensible + def put(self, subtask_id: str, band_name: str, priority: Tuple): + logger.debug(f"Subtask {subtask_id} enqueued in band {band_name}") + item = QueueItem(subtask_id, priority) + self._subtask_id_to_item[subtask_id] = item + self._queues[band_name].put_nowait(item) + + @mo.extensible + def update_priority(self, subtask_id: str, band_name: str, priority: Tuple): + try: + old_item = self._subtask_id_to_item[subtask_id] + except KeyError: + return + + if old_item.priority >= priority: + return + new_item = QueueItem(old_item.subtask_id, priority) + self._subtask_id_to_item[subtask_id] = new_item + self._queues[band_name].put_nowait(new_item) + + async def get(self, band_name: str): + item = None + while True: + try: + item = await self._queues[band_name].get() + + subtask_id = item.subtask_id + if item.subtask_id not in self._subtask_id_to_item: + continue + + await self._semaphores[band_name].acquire() + if item.subtask_id not in self._subtask_id_to_item: + self._semaphores[band_name].release() + continue + slot = self._band_active_slots[band_name].pop() + break + except asyncio.CancelledError: + if item is not None: + # put back enqueued item + self._queues[band_name].put_nowait(item) + raise + + self._subtask_id_to_item.pop(subtask_id) + self._subtask_id_to_band_slot[subtask_id] = (band_name, slot) + return item.subtask_id, slot + + def release_slot(self, subtask_id: str, errors: str = "raise"): + try: + band_name, slot_id = self._subtask_id_to_band_slot.pop(subtask_id) + except KeyError: + if errors == "raise": + raise + else: + return + self._band_active_slots[band_name].append(slot_id) + self._semaphores[band_name].release() + + @mo.extensible + def remove(self, subtask_id: str): + removed = self._subtask_id_to_item.pop(subtask_id, None) + self.release_slot(subtask_id, errors="ignore") + return None if removed is None else subtask_id + + +class SubtaskPrepareQueueActor(SubtaskPriorityQueueActor): + pass + + +class SubtaskExecutionQueueActor(SubtaskPriorityQueueActor): + async def __post_create__(self): + await super().__post_create__() + + from .slotmanager import SlotManagerActor + + self._slot_manager = await mo.actor_ref( + SlotManagerActor.default_uid(), address=self.address + ) + + async def restart_free_slots(self, band_name: str): + slots = [] + sem = self._semaphores[band_name] + + # occupy all free slots + while not sem.locked(): + try: + await asyncio.wait_for(sem.acquire(), timeout=0.1) + slots.append(self._band_active_slots[band_name].pop()) + except asyncio.TimeoutError: + break + + for slot in slots: + await self._slot_manager.kill_slot(band_name, int(slot)) + + for slot in slots: + self._band_active_slots[band_name].append(slot) + sem.release() diff --git a/mars/services/scheduling/worker/quota.py b/mars/services/scheduling/worker/quota.py index 584f05b32d..72140ae01d 100644 --- a/mars/services/scheduling/worker/quota.py +++ b/mars/services/scheduling/worker/quota.py @@ -23,6 +23,7 @@ from .... import oscar as mo from .... import resource as mars_resource from ....typing import BandType +from ....utils import dataslots from ...cluster import QuotaInfo logger = logging.getLogger(__name__) @@ -30,6 +31,7 @@ QuotaDumpType = namedtuple("QuotaDumpType", "allocations requests hold_sizes") +@dataslots @dataclass class QuotaRequest: req_size: Tuple @@ -38,6 +40,10 @@ class QuotaRequest: event: asyncio.Event +class QuotaInsufficientError(Exception): + pass + + class QuotaActor(mo.Actor): @classmethod def gen_uid(cls, band_name: str): @@ -89,11 +95,17 @@ def _report_quota_info(self): self._cluster_api.set_band_quota_info(self._band_name, quota_info) ) - async def request_batch_quota(self, batch: Dict): + async def request_batch_quota(self, batch: Dict, insufficient: str = "enqueue"): """ Request for resources in a batch - :param batch: the request dict in form {request_key: request_size, ...} - :return: if request is returned immediately, return True, otherwise False + + Parameters + ---------- + batch + the request dict in form {request_key: request_size, ...} + insufficient + "enqueue" if quota is not sufficient, otherwise will raise + an QuotaInsufficientError """ all_allocated = True # check if the request is already allocated @@ -130,7 +142,7 @@ async def request_batch_quota(self, batch: Dict): ) await self.alter_allocations(keys, quota_sizes, allocate=True) return - else: + elif insufficient != "raise": # current free space cannot satisfy the request, the request is queued if not has_space: self._log_allocate( @@ -144,6 +156,8 @@ async def request_batch_quota(self, batch: Dict): quota_request = QuotaRequest(quota_sizes, delta, time.time(), event) if keys not in self._requests: self._requests[keys] = quota_request + else: + raise QuotaInsufficientError async def waiter(): try: @@ -312,11 +326,12 @@ async def __post_create__(self): delay=self._refresh_time ) - from .workerslot import BandSlotManagerActor + from .queues import SubtaskExecutionQueueActor try: - self._slot_manager_ref = await mo.actor_ref( - uid=BandSlotManagerActor.gen_uid(self._band[1]), address=self.address + self._execution_queue_ref = await mo.actor_ref( + uid=SubtaskExecutionQueueActor.default_uid(), + address=self.address, ) except mo.ActorNotExist: # pragma: no cover pass @@ -356,9 +371,9 @@ async def _has_space(self, delta: int): self._hard_limit, ) - if self._enable_kill_slot and self._slot_manager_ref is not None: + if self._enable_kill_slot and self._execution_queue_ref is not None: logger.info("Restarting free slots to obtain more memory") - await self._slot_manager_ref.restart_free_slots() + await self._execution_queue_ref.restart_free_slots(self._band_name) return False return await super()._has_space(delta) diff --git a/mars/services/scheduling/worker/service.py b/mars/services/scheduling/worker/service.py index dfd260a345..1ed6e4eaae 100644 --- a/mars/services/scheduling/worker/service.py +++ b/mars/services/scheduling/worker/service.py @@ -15,9 +15,10 @@ from .... import oscar as mo from ....utils import calc_size_by_str from ...core import AbstractService -from .workerslot import WorkerSlotManagerActor +from .slotmanager import SlotManagerActor +from .queues import SubtaskPrepareQueueActor, SubtaskExecutionQueueActor from .quota import WorkerQuotaManagerActor -from .execution import SubtaskExecutionActor, DEFAULT_SUBTASK_MAX_RETRIES +from .exec import SubtaskExecutionActor class SchedulingWorkerService(AbstractService): @@ -50,13 +51,11 @@ async def start(self): scheduling_config.get("mem_hard_limit", "95%"), total_mem ) enable_kill_slot = scheduling_config.get("enable_kill_slot", True) - subtask_max_retries = scheduling_config.get( - "subtask_max_retries", DEFAULT_SUBTASK_MAX_RETRIES - ) + subtask_max_retries = scheduling_config.get("subtask_max_retries", None) await mo.create_actor( - WorkerSlotManagerActor, - uid=WorkerSlotManagerActor.default_uid(), + SlotManagerActor, + uid=SlotManagerActor.default_uid(), address=address, ) await mo.create_actor( @@ -69,6 +68,16 @@ async def start(self): uid=WorkerQuotaManagerActor.default_uid(), address=address, ) + await mo.create_actor( + SubtaskPrepareQueueActor, + uid=SubtaskPrepareQueueActor.default_uid(), + address=address, + ) + await mo.create_actor( + SubtaskExecutionQueueActor, + uid=SubtaskExecutionQueueActor.default_uid(), + address=address, + ) await mo.create_actor( SubtaskExecutionActor, subtask_max_retries=subtask_max_retries, @@ -87,11 +96,19 @@ async def stop(self): ) await mo.destroy_actor( mo.create_actor_ref( - uid=WorkerQuotaManagerActor.default_uid(), address=address + uid=SubtaskExecutionQueueActor.default_uid(), address=address ) ) await mo.destroy_actor( mo.create_actor_ref( - uid=WorkerSlotManagerActor.default_uid(), address=address + uid=SubtaskPrepareQueueActor.default_uid(), address=address ) ) + await mo.destroy_actor( + mo.create_actor_ref( + uid=WorkerQuotaManagerActor.default_uid(), address=address + ) + ) + await mo.destroy_actor( + mo.create_actor_ref(uid=SlotManagerActor.default_uid(), address=address) + ) diff --git a/mars/services/scheduling/worker/slotmanager.py b/mars/services/scheduling/worker/slotmanager.py new file mode 100644 index 0000000000..5f13da128e --- /dev/null +++ b/mars/services/scheduling/worker/slotmanager.py @@ -0,0 +1,52 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from typing import Dict, Tuple + +from .... import oscar as mo +from ....oscar.backends.allocate_strategy import IdleLabel +from ...cluster import ClusterAPI + + +class SlotManagerActor(mo.Actor): + _slot_to_ref: Dict[Tuple[str, int], mo.ActorRef] + + async def __post_create__(self): + cluster_api = await ClusterAPI.create(self.address) + bands = await cluster_api.get_bands() + + self._slot_to_ref = dict() + for band, slot_num in bands.items(): + band_name = band[1] + for slot_id in range(slot_num): + self._slot_to_ref[(band_name, slot_id)] = await mo.create_actor( + SlotControlActor, + uid=SlotControlActor.gen_uid(band_name, slot_id), + allocate_strategy=IdleLabel(band_name, "slot_control"), + address=self.address, + ) + + async def get_slot_address(self, band_name: str, slot_id: int) -> str: + return self._slot_to_ref[(band_name, slot_id)].address + + async def kill_slot(self, band_name: str, slot_id: int): + ref = self._slot_to_ref[(band_name, slot_id)] + await mo.kill_actor(ref) + await mo.wait_actor_pool_recovered(ref.address, self.address) + + +class SlotControlActor(mo.Actor): + @classmethod + def gen_uid(cls, band_name: str, slot_id: int): + return f"{band_name}_{slot_id}slot_control" diff --git a/mars/services/scheduling/worker/tests/test_execution.py b/mars/services/scheduling/worker/tests/test_execution.py deleted file mode 100644 index cce12d3443..0000000000 --- a/mars/services/scheduling/worker/tests/test_execution.py +++ /dev/null @@ -1,493 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import asyncio -import os -import tempfile -import time -import uuid -from contextlib import asynccontextmanager -from typing import Tuple - -import numpy as np -import pandas as pd -import pytest - -from ..... import oscar as mo -from ..... import remote as mr -from .....core import ( - ChunkGraph, - ChunkGraphBuilder, - TileableGraph, - TileableGraphBuilder, - OutputType, -) -from .....remote.core import RemoteFunction -from .....tensor.fetch import TensorFetch -from .....tensor.arithmetic import TensorTreeAdd -from .....utils import Timer -from ....cluster import MockClusterAPI -from ....lifecycle import MockLifecycleAPI -from ....meta import MockMetaAPI -from ....session import MockSessionAPI -from ....storage import MockStorageAPI -from ....storage.handler import StorageHandlerActor -from ....subtask import MockSubtaskAPI, Subtask, SubtaskStatus -from ....task.supervisor.manager import TaskManagerActor -from ....mutable import MockMutableAPI -from ...supervisor import GlobalSlotManagerActor -from ...worker import SubtaskExecutionActor, QuotaActor, BandSlotManagerActor - - -class CancelDetectActorMixin: - @asynccontextmanager - async def _delay_method(self): - delay_fetch_event = getattr(self, "_delay_fetch_event", None) - delay_wait_event = getattr(self, "_delay_wait_event", None) - try: - if delay_fetch_event is not None: - delay_fetch_event.set() - if delay_wait_event is not None: - await delay_wait_event.wait() - yield - except asyncio.CancelledError: - self._is_cancelled = True - raise - - def set_delay_fetch_event( - self, fetch_event: asyncio.Event, wait_event: asyncio.Event - ): - setattr(self, "_delay_fetch_event", fetch_event) - setattr(self, "_delay_wait_event", wait_event) - - def get_is_cancelled(self): - return getattr(self, "_is_cancelled", False) - - -class MockStorageHandlerActor(StorageHandlerActor, CancelDetectActorMixin): - async def fetch_batch(self, *args, **kwargs): - async with self._delay_method(): - return super().fetch_batch(*args, **kwargs) - - -class MockQuotaActor(QuotaActor, CancelDetectActorMixin): - def __init__(self, *args, **kw): - super().__init__(*args, **kw) - self._batch_quota_reqs = [] - - async def request_batch_quota(self, batch): - self._batch_quota_reqs.append(batch) - async with self._delay_method(): - return super().request_batch_quota(batch) - - def get_batch_quota_reqs(self): - return self._batch_quota_reqs - - -class MockBandSlotManagerActor(BandSlotManagerActor, CancelDetectActorMixin): - async def acquire_free_slot(self, session_stid: Tuple[str, str], block=True): - if getattr(self, "_delay_function", None) != "acquire_free_slot": - return super().acquire_free_slot(session_stid, block) - else: - async with self._delay_method(): - return super().acquire_free_slot(session_stid, block) - - async def upload_slot_usages(self, periodical: bool = False): - if ( - getattr(self, "_delay_function", None) != "upload_slot_usages" - or periodical is True - ): - return super().upload_slot_usages(periodical) - else: - async with self._delay_method(): - return super().upload_slot_usages(periodical) - - def set_delay_function(self, name): - self._delay_function = name - - -class MockGlobalSlotManagerActor(GlobalSlotManagerActor, CancelDetectActorMixin): - async def __post_create__(self): - pass - - async def __pre_destroy__(self): - pass - - async def update_subtask_slots( - self, band, session_id: str, subtask_id: str, slots: int - ): - pass - - -class MockTaskManager(mo.Actor): - def __init__(self): - self._results = [] - - def set_subtask_result(self, result): - self._results.append(result) - - def get_results(self): - return self._results - - -@pytest.fixture -async def actor_pool(request): - n_slots, enable_kill = request.param - pool = await mo.create_actor_pool( - "127.0.0.1", labels=[None] + ["numa-0"] * n_slots, n_process=n_slots - ) - - async with pool: - session_id = "test_session" - await MockClusterAPI.create( - pool.external_address, band_to_slots={"numa-0": n_slots} - ) - await MockSessionAPI.create(pool.external_address, session_id=session_id) - meta_api = await MockMetaAPI.create(session_id, pool.external_address) - await MockLifecycleAPI.create(session_id, pool.external_address) - await MockSubtaskAPI.create(pool.external_address) - await MockMutableAPI.create(session_id, pool.external_address) - storage_api = await MockStorageAPI.create( - session_id, - pool.external_address, - storage_handler_cls=MockStorageHandlerActor, - ) - - # create assigner actor - execution_ref = await mo.create_actor( - SubtaskExecutionActor, - subtask_max_retries=0, - enable_kill_slot=enable_kill, - uid=SubtaskExecutionActor.default_uid(), - address=pool.external_address, - ) - # create quota actor - quota_ref = await mo.create_actor( - MockQuotaActor, - "numa-0", - 102400, - uid=QuotaActor.gen_uid("numa-0"), - address=pool.external_address, - ) - # create dispatcher actor - band_slot_ref = await mo.create_actor( - MockBandSlotManagerActor, - (pool.external_address, "numa-0"), - n_slots, - uid=BandSlotManagerActor.gen_uid("numa-0"), - address=pool.external_address, - ) - - # create global slot manager actor - global_slot_ref = await mo.create_actor( - MockGlobalSlotManagerActor, - uid=GlobalSlotManagerActor.default_uid(), - address=pool.external_address, - ) - - # create mock task manager actor - task_manager_ref = await mo.create_actor( - MockTaskManager, - uid=TaskManagerActor.gen_uid(session_id), - address=pool.external_address, - ) - - try: - yield pool, session_id, meta_api, storage_api, execution_ref - finally: - await mo.destroy_actor(task_manager_ref) - await mo.destroy_actor(band_slot_ref) - await mo.destroy_actor(global_slot_ref) - await mo.destroy_actor(quota_ref) - await mo.destroy_actor(execution_ref) - await MockStorageAPI.cleanup(pool.external_address) - await MockSubtaskAPI.cleanup(pool.external_address) - await MockClusterAPI.cleanup(pool.external_address) - await MockMutableAPI.cleanup(session_id, pool.external_address) - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [(1, True)], indirect=True) -async def test_execute_tensor(actor_pool): - pool, session_id, meta_api, storage_api, execution_ref = actor_pool - - data1 = np.random.rand(10, 10) - data2 = np.random.rand(10, 10) - - input1 = TensorFetch( - key="input1", source_key="input2", dtype=np.dtype(int) - ).new_chunk([]) - input2 = TensorFetch( - key="input2", source_key="input2", dtype=np.dtype(int) - ).new_chunk([]) - result_chunk = TensorTreeAdd(args=[input1, input2]).new_chunk( - [input1, input2], shape=data1.shape, dtype=data1.dtype - ) - - await meta_api.set_chunk_meta( - input1, - memory_size=data1.nbytes, - store_size=data1.nbytes, - bands=[(pool.external_address, "numa-0")], - ) - await meta_api.set_chunk_meta( - input2, - memory_size=data1.nbytes, - store_size=data2.nbytes, - bands=[(pool.external_address, "numa-0")], - ) - # todo use different storage level when storage ready - await storage_api.put(input1.key, data1) - await storage_api.put(input2.key, data2) - - chunk_graph = ChunkGraph([result_chunk]) - chunk_graph.add_node(input1) - chunk_graph.add_node(input2) - chunk_graph.add_node(result_chunk) - chunk_graph.add_edge(input1, result_chunk) - chunk_graph.add_edge(input2, result_chunk) - - subtask = Subtask("test_subtask", session_id=session_id, chunk_graph=chunk_graph) - await execution_ref.run_subtask(subtask, "numa-0", pool.external_address) - - # check if results are correct - result = await storage_api.get(result_chunk.key) - np.testing.assert_array_equal(data1 + data2, result) - - # check if quota computations are correct - quota_ref = await mo.actor_ref( - QuotaActor.gen_uid("numa-0"), address=pool.external_address - ) - [quota] = await quota_ref.get_batch_quota_reqs() - assert quota[(subtask.session_id, subtask.subtask_id)] == data1.nbytes - - # check if metas are correct - result_meta = await meta_api.get_chunk_meta(result_chunk.key) - assert result_meta["object_id"] == result_chunk.key - assert result_meta["shape"] == result.shape - - -_cancel_phases = [ - "prepare", - "quota", - "slot", - "execute", - "finally", - "immediately", -] - - -@pytest.mark.asyncio -@pytest.mark.parametrize( - "actor_pool,cancel_phase", - [((1, True), phase) for phase in _cancel_phases], - indirect=["actor_pool"], -) -async def test_execute_with_cancel(actor_pool, cancel_phase): - pool, session_id, meta_api, storage_api, execution_ref = actor_pool - delay_fetch_event = asyncio.Event() - delay_wait_event = asyncio.Event() - - # config for different phases - ref_to_delay = None - if cancel_phase == "prepare": - ref_to_delay = await mo.actor_ref( - StorageHandlerActor.gen_uid("numa-0"), address=pool.external_address - ) - elif cancel_phase == "quota": - ref_to_delay = await mo.actor_ref( - QuotaActor.gen_uid("numa-0"), address=pool.external_address - ) - elif cancel_phase == "slot": - ref_to_delay = await mo.actor_ref( - BandSlotManagerActor.gen_uid("numa-0"), address=pool.external_address - ) - await ref_to_delay.set_delay_function("acquire_free_slot") - elif cancel_phase == "finally": - ref_to_delay = await mo.actor_ref( - BandSlotManagerActor.gen_uid("numa-0"), address=pool.external_address - ) - await ref_to_delay.set_delay_function("upload_slot_usages") - if ref_to_delay: - await ref_to_delay.set_delay_fetch_event(delay_fetch_event, delay_wait_event) - else: - delay_fetch_event.set() - - def delay_fun(delay, _inp1): - if not ref_to_delay: - time.sleep(delay) - return (delay,) - - input1 = TensorFetch( - key="input1", source_key="input1", dtype=np.dtype(int) - ).new_chunk([]) - remote_result = RemoteFunction( - function=delay_fun, function_args=[100, input1], function_kwargs={}, n_output=1 - ).new_chunk([input1]) - - data1 = np.random.rand(10, 10) - await meta_api.set_chunk_meta( - input1, - memory_size=data1.nbytes, - store_size=data1.nbytes, - bands=[(pool.external_address, "numa-0")], - ) - await storage_api.put(input1.key, data1) - - chunk_graph = ChunkGraph([remote_result]) - chunk_graph.add_node(input1) - chunk_graph.add_node(remote_result) - chunk_graph.add_edge(input1, remote_result) - - subtask = Subtask( - f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph - ) - aiotask = asyncio.create_task( - execution_ref.run_subtask(subtask, "numa-0", pool.external_address) - ) - if ref_to_delay: - await delay_fetch_event.wait() - else: - if cancel_phase != "immediately": - await asyncio.sleep(1) - - with Timer() as timer: - await asyncio.wait_for( - execution_ref.cancel_subtask(subtask.subtask_id, kill_timeout=1), - timeout=30, - ) - r = await asyncio.wait_for(aiotask, timeout=30) - assert r.status == SubtaskStatus.cancelled - assert timer.duration < 15 - - # check for different phases - if ref_to_delay is not None: - assert await ref_to_delay.get_is_cancelled() - delay_wait_event.set() - - # test if slot is restored - remote_tileable = mr.spawn(delay_fun, args=(0.5, None)) - graph = TileableGraph([remote_tileable.data]) - next(TileableGraphBuilder(graph).build()) - - chunk_graph = next(ChunkGraphBuilder(graph, fuse_enabled=False).build()) - - subtask = Subtask( - f"test_subtask2_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph - ) - await asyncio.wait_for( - execution_ref.run_subtask(subtask, "numa-0", pool.external_address), timeout=30 - ) - - -def test_estimate_size(): - from ..execution import SubtaskExecutionActor - from .....dataframe.arithmetic import DataFrameAdd - from .....dataframe.fetch import DataFrameFetch - from .....dataframe.utils import parse_index - - index_value = parse_index(pd.Index([10, 20, 30], dtype=np.int64)) - - input1 = DataFrameFetch(output_types=[OutputType.series],).new_chunk( - [], _key="INPUT1", shape=(np.nan,), dtype=np.dtype("O"), index_value=index_value - ) - input2 = DataFrameFetch(output_types=[OutputType.series],).new_chunk( - [], _key="INPUT2", shape=(np.nan,), dtype=np.dtype("O"), index_value=index_value - ) - result_chunk = DataFrameAdd( - axis=0, output_types=[OutputType.series], lhs=input1, rhs=input2 - ).new_chunk( - [input1, input2], - _key="ADD_RESULT", - shape=(np.nan,), - dtype=np.dtype("O"), - index_value=index_value, - ) - - chunk_graph = ChunkGraph([result_chunk]) - chunk_graph.add_node(input1) - chunk_graph.add_node(input2) - chunk_graph.add_node(result_chunk) - chunk_graph.add_edge(input1, result_chunk) - chunk_graph.add_edge(input2, result_chunk) - - input_sizes = { - "INPUT1": (1024, 1024), - "INPUT2": (1024, 1024), - } - - subtask = Subtask("test_subtask", session_id="session_id", chunk_graph=chunk_graph) - result = SubtaskExecutionActor._estimate_sizes(subtask, input_sizes) - assert result[0] == 1024 - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [(1, False)], indirect=True) -async def test_cancel_without_kill(actor_pool): - pool, session_id, meta_api, storage_api, execution_ref = actor_pool - executed_file = os.path.join( - tempfile.gettempdir(), f"mars_test_cancel_without_kill_{os.getpid()}.tmp" - ) - - def delay_fun(delay): - import mars - - open(executed_file, "w").close() - time.sleep(delay) - mars._slot_marker = 1 - return delay - - def check_fun(): - import mars - - return getattr(mars, "_slot_marker", False) - - remote_result = RemoteFunction( - function=delay_fun, function_args=[2], function_kwargs={} - ).new_chunk([]) - chunk_graph = ChunkGraph([remote_result]) - chunk_graph.add_node(remote_result) - - subtask = Subtask( - f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph - ) - aiotask = asyncio.create_task( - execution_ref.run_subtask(subtask, "numa-0", pool.external_address) - ) - await asyncio.sleep(0.5) - - await asyncio.wait_for( - execution_ref.cancel_subtask(subtask.subtask_id, kill_timeout=1), - timeout=30, - ) - r = await asyncio.wait_for(aiotask, timeout=30) - assert r.status == SubtaskStatus.cancelled - - remote_result = RemoteFunction( - function=check_fun, function_args=[], function_kwargs={} - ).new_chunk([]) - chunk_graph = ChunkGraph([remote_result]) - chunk_graph.add_node(remote_result) - - subtask = Subtask( - f"test_subtask_{uuid.uuid4()}", session_id=session_id, chunk_graph=chunk_graph - ) - await asyncio.wait_for( - execution_ref.run_subtask(subtask, "numa-0", pool.external_address), timeout=30 - ) - - # check if slots not killed (or slot assignment may be cancelled) - if os.path.exists(executed_file): - assert await storage_api.get(remote_result.key) - os.unlink(executed_file) diff --git a/mars/services/scheduling/worker/tests/test_quota.py b/mars/services/scheduling/worker/tests/test_quota.py index b011b77c06..044b42b94a 100644 --- a/mars/services/scheduling/worker/tests/test_quota.py +++ b/mars/services/scheduling/worker/tests/test_quota.py @@ -23,16 +23,16 @@ from ..... import oscar as mo from .....tests.core import mock from .....utils import get_next_port -from ...worker import QuotaActor, MemQuotaActor, BandSlotManagerActor +from .. import QuotaActor, MemQuotaActor, SubtaskExecutionQueueActor QuotaActorRef = Union[QuotaActor, mo.ActorRef] -class MockBandSlotManagerActor(mo.Actor): +class MockSubtaskExecutionQueueActor(mo.Actor): def get_restart_record(self): return getattr(self, "_restart_record", False) - def restart_free_slots(self): + def restart_free_slots(self, band_name: str): self._restart_record = True @@ -146,9 +146,9 @@ async def test_mem_quota_allocation(actor_pool, enable_kill_slot): from .....utils import AttributeDict mock_mem_stat = AttributeDict(dict(total=300, available=50, used=0, free=50)) - mock_band_slot_manager_ref = await mo.create_actor( - MockBandSlotManagerActor, - uid=BandSlotManagerActor.gen_uid("numa-0"), + mock_execution_queue_ref = await mo.create_actor( + MockSubtaskExecutionQueueActor, + uid=SubtaskExecutionQueueActor.default_uid(), address=actor_pool.external_address, ) quota_ref = await mo.create_actor( @@ -179,6 +179,6 @@ async def task_fun(): await asyncio.wait_for(task, timeout=1) assert 0.15 < abs(time_recs[0] - time_recs[1]) < 1 assert ( - bool(await mock_band_slot_manager_ref.get_restart_record()) + bool(await mock_execution_queue_ref.get_restart_record()) == enable_kill_slot ) diff --git a/mars/services/scheduling/worker/tests/test_workerslot.py b/mars/services/scheduling/worker/tests/test_workerslot.py deleted file mode 100644 index c7f0a1e44a..0000000000 --- a/mars/services/scheduling/worker/tests/test_workerslot.py +++ /dev/null @@ -1,346 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import asyncio -import os -import sys -import time -from typing import Tuple, Union - -import psutil -import pytest -import pandas as pd - -from ..... import oscar as mo -from .....oscar import ServerClosed -from .....oscar.errors import NoFreeSlot, SlotStateError -from .....oscar.backends.allocate_strategy import IdleLabel -from .....utils import get_next_port -from ...supervisor import GlobalSlotManagerActor -from ...worker import BandSlotManagerActor, BandSlotControlActor - - -class MockGlobalSlotManagerActor(mo.Actor): - def __init__(self): - self._result = None - - @mo.extensible - def update_subtask_slots( - self, band: Tuple, session_id: str, subtask_id: str, slots: int - ): - self._result = (band, session_id, subtask_id, slots) - - def get_result(self): - return self._result - - -@pytest.fixture -async def actor_pool(request): - start_method = ( - os.environ.get("POOL_START_METHOD", "forkserver") - if sys.platform != "win32" - else None - ) - n_slots = request.param - pool = await mo.create_actor_pool( - f"127.0.0.1:{get_next_port()}", - n_process=n_slots, - labels=[None] + ["numa-0"] * n_slots, - subprocess_start_method=start_method, - ) - - async with pool: - global_slots_ref = await mo.create_actor( - MockGlobalSlotManagerActor, - uid=GlobalSlotManagerActor.default_uid(), - address=pool.external_address, - ) - slot_manager_ref = await mo.create_actor( - BandSlotManagerActor, - (pool.external_address, "numa-0"), - n_slots, - global_slots_ref, - uid=BandSlotManagerActor.gen_uid("numa-0"), - address=pool.external_address, - ) - try: - yield pool, slot_manager_ref - finally: - await slot_manager_ref.destroy() - - -ActorPoolType = Tuple[mo.MainActorPoolType, Union[BandSlotManagerActor, mo.ActorRef]] - - -class TaskActor(mo.Actor): - def __init__(self, call_logs, slot_id=0): - self._call_logs = call_logs - self._dispatch_ref = None - self._slot_id = slot_id - - @classmethod - def gen_uid(cls, slot_id): - return f"{slot_id}_task_actor" - - async def __post_create__(self): - self._dispatch_ref = await mo.actor_ref( - BandSlotManagerActor.gen_uid("numa-0"), address=self.address - ) - await self._dispatch_ref.register_slot.tell(self._slot_id, os.getpid()) - - async def queued_call(self, key, session_stid, delay): - try: - self._call_logs[key] = time.time() - await asyncio.sleep(delay) - finally: - if session_stid is not None: - await self._dispatch_ref.release_free_slot(self._slot_id, session_stid) - - def get_call_logs(self): - return self._call_logs - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [0], indirect=True) -async def test_slot_assign(actor_pool: ActorPoolType): - pool, slot_manager_ref = actor_pool - - call_logs = dict() - group_size = 4 - delay = 1 - await asyncio.gather( - *( - mo.create_actor( - TaskActor, - call_logs, - slot_id=slot_id, - uid=TaskActor.gen_uid(slot_id), - address=pool.external_address, - ) - for slot_id in range(group_size) - ) - ) - assert len((await slot_manager_ref.dump_data()).free_slots) == group_size - - async def task_fun(idx): - session_stid = ("session_id", f"subtask_id{idx}") - slot_id = await slot_manager_ref.acquire_free_slot(session_stid) - assert slot_id == await slot_manager_ref.get_subtask_slot(session_stid) - ref = await mo.actor_ref( - uid=TaskActor.gen_uid(slot_id), address=pool.external_address - ) - await ref.queued_call(idx, session_stid, delay) - - tasks = [] - start_time = time.time() - for idx in range(group_size + 1): - tasks.append(asyncio.create_task(task_fun(idx))) - await asyncio.gather(*tasks) - - log_series = pd.Series(call_logs).sort_index() - start_time - assert len(log_series) == group_size + 1 - assert log_series.iloc[:group_size].max() < delay / 4 - assert log_series.iloc[group_size:].min() > delay / 4 - - call_logs.clear() - tasks = [] - start_time = time.time() - for idx in range(group_size * 2 + 1): - tasks.append(asyncio.create_task(task_fun(idx))) - await asyncio.sleep(delay / 10) - tasks[group_size].cancel() - await asyncio.wait(tasks) - - with pytest.raises(asyncio.CancelledError): - tasks[group_size].result() - - log_series = pd.Series(call_logs).sort_index() - start_time - - assert len(log_series) == group_size * 2 - assert log_series.iloc[:group_size].max() < delay / 4 - assert log_series.iloc[group_size:].min() > delay / 4 - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [1], indirect=True) -async def test_slot_kill(actor_pool: ActorPoolType): - pool, slot_manager_ref = actor_pool - - strategy = IdleLabel("numa-0", "task_actor") - task_ref = await mo.create_actor( - TaskActor, {}, allocate_strategy=strategy, address=pool.external_address - ) - - assert await mo.actor_ref( - BandSlotControlActor.gen_uid("numa-0", 0), address=pool.external_address - ) - delayed_task = asyncio.create_task(task_ref.queued_call("key", None, 10)) - await asyncio.sleep(0.1) - - # check if process hosting the actor is closed - kill_task = asyncio.create_task(slot_manager_ref.kill_slot(0)) - await asyncio.sleep(0) - kill_task2 = asyncio.create_task(slot_manager_ref.kill_slot(0)) - - with pytest.raises(ServerClosed): - await delayed_task - - # check if slot actor is restored - await kill_task - # check if secondary task makes no change - await kill_task2 - - assert await mo.actor_ref( - BandSlotControlActor.gen_uid("numa-0", 0), address=pool.external_address - ) - assert await mo.actor_ref(task_ref) - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [3], indirect=True) -async def test_slot_restart(actor_pool: ActorPoolType): - pool, slot_manager_ref = actor_pool - - strategy = IdleLabel("numa-0", "task_actor") - task_refs = [] - for idx in range(3): - ref = await mo.create_actor( - TaskActor, - {}, - slot_id=idx, - allocate_strategy=strategy, - address=pool.external_address, - ) - await ref.queued_call("idx", None, idx) - task_refs.append(ref) - - await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id1")) - slot_id2 = await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id2")) - await slot_manager_ref.release_free_slot(slot_id2, ("session_id", "subtask_id2")) - - async def record_finish_time(coro): - await coro - return time.time() - - restart_task1 = asyncio.create_task( - record_finish_time(slot_manager_ref.restart_free_slots()) - ) - await asyncio.sleep(0) - restart_task2 = asyncio.create_task( - record_finish_time(slot_manager_ref.restart_free_slots()) - ) - acquire_task = asyncio.create_task( - record_finish_time( - slot_manager_ref.acquire_free_slot(("session_id", "subtask_id3")) - ) - ) - - await asyncio.gather(restart_task1, restart_task2, acquire_task) - - # check only slots with running records are restarted - assert len(await task_refs[0].get_call_logs()) > 0 - assert len(await task_refs[1].get_call_logs()) == 0 - assert len(await task_refs[2].get_call_logs()) > 0 - - assert abs(restart_task1.result() - acquire_task.result()) < 0.1 - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [1], indirect=True) -async def test_report_usage(actor_pool: ActorPoolType): - pool, slot_manager_ref = actor_pool - - await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id")) - await asyncio.sleep(1.3) - - global_slot_ref = await mo.actor_ref( - uid=GlobalSlotManagerActor.default_uid(), address=pool.external_address - ) - _band, session_id, subtask_id, slots = await global_slot_ref.get_result() - assert slots == pytest.approx(1.0) - assert session_id == "session_id" - assert subtask_id == "subtask_id" - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [1], indirect=True) -async def test_slot_fault_tolerance(actor_pool: ActorPoolType): - pool, slot_manager_ref = actor_pool - # acquire -> slot restarted = can't acquire more. - slot_id = await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id")) - await slot_manager_ref.register_slot(slot_id, os.getpid()) - with pytest.raises(NoFreeSlot): - await slot_manager_ref.acquire_free_slot( - ("session_id", "subtask_id"), block=False - ) - await slot_manager_ref.release_free_slot(slot_id, ("session_id", "subtask_id")) - - # acquire -> release -> slot restarted = can only acquire once. - slot_id = await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id2")) - await slot_manager_ref.release_free_slot(slot_id, ("session_id", "subtask_id2")) - await slot_manager_ref.register_slot(slot_id, os.getpid()) - await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id2")) - with pytest.raises(NoFreeSlot): - await slot_manager_ref.acquire_free_slot( - ("session_id", "subtask_id2"), block=False - ) - await slot_manager_ref.release_free_slot(slot_id, ("session_id", "subtask_id2")) - - # acquire -> release -> acquire -> slot restarted = can't acquire more. - slot_id = await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id3")) - await slot_manager_ref.release_free_slot(slot_id, ("session_id", "subtask_id3")) - await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id3")) - await slot_manager_ref.register_slot(slot_id, os.getpid()) - with pytest.raises(NoFreeSlot): - await slot_manager_ref.acquire_free_slot( - ("session_id", "subtask_id3"), block=False - ) - await slot_manager_ref.release_free_slot(slot_id, ("session_id", "subtask_id3")) - - -@pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [1], indirect=True) -async def test_slot_exception(actor_pool: ActorPoolType): - pool, slot_manager_ref = actor_pool - - # make sure the BandSlotControlActor has registered. - slot_id = await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id")) - await slot_manager_ref.release_free_slot(slot_id, ("session_id", "subtask_id")) - - if sys.platform == "win32": - with pytest.raises(ValueError): - await slot_manager_ref.register_slot(1, -1) - else: - with pytest.raises((psutil.AccessDenied, psutil.NoSuchProcess)): - await slot_manager_ref.register_slot(1, 0) - - dump_data = await slot_manager_ref.dump_data() - # after the register_slot is correctly handled, - # we can assert 1 not in free slots. - assert 1 in dump_data.free_slots - - slot_id = await slot_manager_ref.acquire_free_slot(("session_id", "subtask_id")) - with pytest.raises(SlotStateError): - # release session_stid not matched the acquired value. - await slot_manager_ref.release_free_slot(slot_id, ("session_id", "subtask_id1")) - - dump_data = await slot_manager_ref.dump_data() - # the slot is not released. - assert slot_id not in dump_data.free_slots - - not_acquired_slot = next(iter(dump_data.free_slots)) - with pytest.raises(SlotStateError): - await slot_manager_ref.release_free_slot( - not_acquired_slot, ("session_id", "subtask_id1") - ) diff --git a/mars/services/scheduling/worker/workerslot.py b/mars/services/scheduling/worker/workerslot.py deleted file mode 100644 index 81e6d76665..0000000000 --- a/mars/services/scheduling/worker/workerslot.py +++ /dev/null @@ -1,313 +0,0 @@ -# Copyright 1999-2021 Alibaba Group Holding Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import asyncio -import logging -import os -import time -from typing import Dict, List, NamedTuple, Set, Tuple - -import psutil - -from .... import oscar as mo -from ....oscar.errors import NoFreeSlot, SlotStateError -from ....oscar.backends.allocate_strategy import IdleLabel -from ....typing import BandType -from ...cluster import WorkerSlotInfo, ClusterAPI - -logger = logging.getLogger(__name__) - - -class DispatchDumpType(NamedTuple): - free_slots: Set - fresh_slots: Set - - -class WorkerSlotManagerActor(mo.Actor): - _band_slot_infos: Dict[str, List[WorkerSlotInfo]] - - def __init__(self): - self._cluster_api = None - self._global_slots_ref = None - - self._band_slot_managers = dict() # type: Dict[str, mo.ActorRef] - - async def __post_create__(self): - self._cluster_api = await ClusterAPI.create(self.address) - - band_to_slots = await self._cluster_api.get_bands() - for band, n_slot in band_to_slots.items(): - self._band_slot_managers[band] = await mo.create_actor( - BandSlotManagerActor, - band, - n_slot, - self._global_slots_ref, - uid=BandSlotManagerActor.gen_uid(band[1]), - address=self.address, - ) - - async def __pre_destroy__(self): - await asyncio.gather( - *[mo.destroy_actor(ref) for ref in self._band_slot_managers.values()] - ) - - -class BandSlotManagerActor(mo.Actor): - _free_slots: Set[int] - _fresh_slots: Set[int] - - @classmethod - def gen_uid(cls, band_name: str): - return f"{band_name}_band_slot_manager" - - def __init__( - self, band: BandType, n_slots: int, global_slots_ref: mo.ActorRef = None - ): - super().__init__() - self._cluster_api = None - - self._band = band - self._band_name = band[1] - self._global_slots_ref = global_slots_ref - self._n_slots = n_slots - - self._semaphore = asyncio.Semaphore(0) - self._slot_control_refs = dict() - self._free_slots = set() - self._fresh_slots = set() - self._slot_kill_events = dict() - - self._restarting = False - self._restart_done_event = asyncio.Event() - - self._session_stid_to_slot = dict() - self._slot_to_session_stid = dict() - self._last_report_time = time.time() - - self._slot_to_proc = dict() - self._usage_upload_task = None - - async def __post_create__(self): - try: - self._cluster_api = await ClusterAPI.create(self.address) - except mo.ActorNotExist: - pass - - strategy = IdleLabel(self._band_name, f"worker_slot_control") - for slot_id in range(self._n_slots): - self._slot_control_refs[slot_id] = await mo.create_actor( - BandSlotControlActor, - self.ref(), - self._band_name, - slot_id, - uid=BandSlotControlActor.gen_uid(self._band_name, slot_id), - address=self.address, - allocate_strategy=strategy, - ) - self._fresh_slots.add(slot_id) - - self._usage_upload_task = self.ref().upload_slot_usages.tell_delay( - periodical=True, delay=1 - ) - - async def __pre_destroy__(self): - self._usage_upload_task.cancel() - - async def _get_global_slot_ref(self): - if self._global_slots_ref is not None: - return self._global_slots_ref - - from ..supervisor import GlobalSlotManagerActor - - [self._global_slots_ref] = await self._cluster_api.get_supervisor_refs( - [GlobalSlotManagerActor.default_uid()] - ) - return self._global_slots_ref - - def get_slot_address(self, slot_id: int): - return self._slot_control_refs[slot_id].address - - def get_subtask_slot(self, session_stid: Tuple[str, str]): - return self._session_stid_to_slot.get(session_stid) - - async def acquire_free_slot(self, session_stid: Tuple[str, str], block=True): - if not block and self._semaphore.locked(): - raise NoFreeSlot(f"No free slot for {session_stid}") - yield self._semaphore.acquire() - if self._restarting: - yield self._restart_done_event.wait() - - slot_id = self._free_slots.pop() - self._fresh_slots.difference_update([slot_id]) - self._slot_to_session_stid[slot_id] = session_stid - self._session_stid_to_slot[session_stid] = slot_id - logger.debug("Slot %d acquired for subtask %r", slot_id, session_stid) - raise mo.Return(slot_id) - - def release_free_slot(self, slot_id: int, session_stid: Tuple[str, str]): - acquired_session_stid = self._slot_to_session_stid.pop(slot_id, None) - if acquired_session_stid is None: - raise SlotStateError(f"Slot {slot_id} is not acquired.") - if acquired_session_stid != session_stid: - raise SlotStateError( - f"Slot {slot_id} releasing state incorrect, " - f"the acquired session_stid: {acquired_session_stid}, " - f"the releasing session_stid: {session_stid}" - ) - acquired_slot_id = self._session_stid_to_slot.pop(acquired_session_stid) - assert acquired_slot_id == slot_id - - logger.debug("Slot %d released", slot_id) - - if slot_id not in self._free_slots: - self._free_slots.add(slot_id) - self._semaphore.release() - - def register_slot(self, slot_id: int, pid: int): - try: - self._fresh_slots.add(slot_id) - if slot_id in self._slot_kill_events: - event = self._slot_kill_events.pop(slot_id) - event.set() - if slot_id in self._slot_to_session_stid: - # We should release the slot by one role, if the slot is - # acquired by the SubtaskExecutionActor, then the slot - # should be released by it, too. - session_stid = self._slot_to_session_stid[slot_id] - logger.info( - "Slot %s registered by pid %s, current acquired session_stid is %s", - slot_id, - pid, - session_stid, - ) - else: - if slot_id not in self._free_slots: - self._free_slots.add(slot_id) - self._semaphore.release() - finally: - # psutil may raises exceptions, but currently we can't handle the register exception, - # so put it to the finally. - # TODO(fyrestone): handle register_slot failure. - self._slot_to_proc[slot_id] = proc = psutil.Process(pid) - # collect initial stats for the process - proc.cpu_percent(interval=None) - - async def _kill_slot(self, slot_id: int): - if slot_id in self._slot_kill_events: - await self._slot_kill_events[slot_id].wait() - return - - event = self._slot_kill_events[slot_id] = asyncio.Event() - # TODO(fyrestone): Make it more reliable. e.g. kill_actor - # success but the actor does not restart. - try: - await mo.kill_actor(self._slot_control_refs[slot_id]) - except ConnectionError: - pass - await event.wait() - - async def kill_slot(self, slot_id: int): - self._free_slots.difference_update([slot_id]) - yield self._kill_slot(slot_id) - - async def restart_free_slots(self): - if self._restarting: - yield self._restart_done_event.wait() - return - - self._restart_done_event = asyncio.Event() - self._restarting = True - slot_ids = [ - slot_id for slot_id in self._free_slots if slot_id not in self._fresh_slots - ] - if slot_ids: - yield asyncio.gather(*[self._kill_slot(slot_id) for slot_id in slot_ids]) - logger.info("%d idle slots restarted", len(slot_ids)) - - self._restarting = False - self._restart_done_event.set() - - async def upload_slot_usages(self, periodical: bool = False): - delays = [] - slot_infos = [] - global_slots_ref = await self._get_global_slot_ref() - for slot_id, proc in self._slot_to_proc.items(): - if slot_id not in self._slot_to_session_stid: - continue - session_id, subtask_id = self._slot_to_session_stid[slot_id] - - try: - usage = proc.cpu_percent(interval=None) / 100.0 - except psutil.NoSuchProcess: # pragma: no cover - continue - - slot_infos.append( - WorkerSlotInfo( - slot_id=slot_id, - session_id=session_id, - subtask_id=subtask_id, - processor_usage=usage, - ) - ) - - if global_slots_ref is not None: # pragma: no branch - # FIXME fix band slot mistake - delays.append( - global_slots_ref.update_subtask_slots.delay( - self._band[1], session_id, subtask_id, max(1.0, usage) - ) - ) - - if delays: # pragma: no branch - yield global_slots_ref.update_subtask_slots.batch(*delays) - if self._cluster_api is not None: - await self._cluster_api.set_band_slot_infos(self._band_name, slot_infos) - - if periodical: - self._usage_upload_task = self.ref().upload_slot_usages.tell_delay( - periodical=True, delay=1 - ) - - def dump_data(self): - """ - Get all refs of slots of a queue - """ - return DispatchDumpType(self._free_slots, self._fresh_slots) - - -class BandSlotControlActor(mo.Actor): - @classmethod - def gen_uid(cls, band_name: str, slot_id: int): - return f"{band_name}_{slot_id}_band_slot_control" - - def __init__(self, manager_ref, band_name, slot_id: int): - self._manager_ref = manager_ref - self._band_name = band_name - self._slot_id = slot_id - self._report_task = None - - async def __post_create__(self): - self._report_task = asyncio.create_task(self._report_slot_ready()) - - async def _report_slot_ready(self): - from ...cluster.api import ClusterAPI - - try: - self._cluster_api = await ClusterAPI.create(self.address) - await self._cluster_api.wait_node_ready() - except mo.ActorNotExist: - pass - - await mo.wait_actor_pool_recovered(self.address) - await self._manager_ref.register_slot.tell(self._slot_id, os.getpid()) diff --git a/mars/services/storage/api/oscar.py b/mars/services/storage/api/oscar.py index 91708f332a..2d3354e750 100644 --- a/mars/services/storage/api/oscar.py +++ b/mars/services/storage/api/oscar.py @@ -118,13 +118,16 @@ async def batch_put(self, args_list, kwargs_list): return await self._storage_handler_ref.put.batch(*puts) @mo.extensible - async def get_infos(self, data_key: str) -> List[DataInfo]: + async def get_infos(self, data_key: str, error: str = "raise") -> List[DataInfo]: """ Get data information items for specific data key Parameters ---------- data_key + object key to obtain info + error: str + raise or ignore Returns ------- @@ -132,7 +135,7 @@ async def get_infos(self, data_key: str) -> List[DataInfo]: List of information for specified key """ return await self._data_manager_ref.get_data_infos( - self._session_id, data_key, self._band_name + self._session_id, data_key, self._band_name, error=error ) @mo.extensible diff --git a/mars/services/subtask/api.py b/mars/services/subtask/api.py index b92e79fe4d..f6df21a366 100644 --- a/mars/services/subtask/api.py +++ b/mars/services/subtask/api.py @@ -15,6 +15,7 @@ from ... import oscar as mo from ...lib.aio import alru_cache from ...oscar.backends.message import ProfilingContext +from ..core import ActorCallback from .core import Subtask @@ -42,7 +43,14 @@ async def _get_subtask_processor_ref(self, session_id: str, slot_address: str): SubtaskProcessorActor.gen_uid(session_id), address=slot_address ) - async def run_subtask_in_slot(self, band_name: str, slot_id: int, subtask: Subtask): + async def run_subtask_in_slot( + self, + band_name: str, + slot_id: int, + subtask: Subtask, + forward_successors: bool = False, + finish_callback: ActorCallback = None, + ): """ Run subtask in current worker @@ -51,6 +59,8 @@ async def run_subtask_in_slot(self, band_name: str, slot_id: int, subtask: Subta band_name subtask slot_id + forward_successors + finish_callback Returns ------- @@ -64,7 +74,9 @@ async def run_subtask_in_slot(self, band_name: str, slot_id: int, subtask: Subta else None ) return await ref.run_subtask.options(profiling_context=profiling_context).send( - subtask + subtask, + forward_successors=forward_successors, + finish_callback=finish_callback, ) async def cancel_subtask_in_slot(self, band_name: str, slot_id: int): diff --git a/mars/services/subtask/core.py b/mars/services/subtask/core.py index 87f7bc5d66..c48292301c 100644 --- a/mars/services/subtask/core.py +++ b/mars/services/subtask/core.py @@ -74,6 +74,7 @@ def __init__( virtual: bool = False, retryable: bool = True, rerun_time: int = 0, + forward_successors: bool = False, extra_config: dict = None, ): super().__init__( @@ -87,6 +88,7 @@ def __init__( virtual=virtual, retryable=retryable, rerun_time=rerun_time, + forward_successors=forward_successors, extra_config=extra_config, ) diff --git a/mars/services/subtask/worker/processor.py b/mars/services/subtask/worker/processor.py index bc9c3de188..c4da83a4ed 100644 --- a/mars/services/subtask/worker/processor.py +++ b/mars/services/subtask/worker/processor.py @@ -32,6 +32,7 @@ from ....typing import BandType from ....utils import get_chunk_key_to_data_keys from ...context import ThreadedServiceContext +from ...core import ActorCallback from ...meta.api import MetaAPI from ...storage import StorageAPI from ...session import SessionAPI @@ -60,6 +61,8 @@ def __init__( band: BandType, supervisor_address: str, engines: List[str] = None, + forward_successors: bool = False, + finish_callback: Optional[ActorCallback] = None, ): self.subtask = subtask self._session_id = self.subtask.session_id @@ -74,6 +77,8 @@ def __init__( self._band = band self._supervisor_address = supervisor_address self._engines = engines if engines is not None else task_options.runtime_engines + self._forward_successors = forward_successors + self._finish_callback = finish_callback # result self.result = SubtaskResult( @@ -84,6 +89,7 @@ def __init__( bands=[self._band], progress=0.0, ) + self.is_released = False self.is_done = asyncio.Event() # status and intermediate states @@ -389,7 +395,7 @@ async def _store_meta( if chunk.key not in data_key_to_store_size: # mapper, set meta, so that storage can make sure # this operand is executed, some sub key is absent - # due to it's empty actually + # due to its empty actually set_chunk_metas.append( self._meta_api.set_chunk_meta.delay( chunk, memory_size=0, store_size=0, bands=[self._band] @@ -441,6 +447,11 @@ async def done(self): self.result.progress = 1.0 self.is_done.set() + async def _release_scheduling(self): + if self._finish_callback is not None: + await self._finish_callback() + self.is_released = True + async def run(self): self.result.status = SubtaskStatus.running input_keys = None @@ -459,6 +470,12 @@ async def run(self): # unpin inputs data unpinned = True await self._unpin_data(input_keys) + + # if we need to forward successors of subtasks, + # data should be stored first + if not self._forward_successors: + await self._release_scheduling() + # store results data ( stored_keys, @@ -466,6 +483,10 @@ async def run(self): memory_sizes, data_key_to_object_id, ) = await self._store_data(chunk_graph) + + if self._forward_successors: + await self._release_scheduling() + # store meta await self._store_meta( chunk_graph, @@ -519,7 +540,6 @@ class SubtaskProcessorActor(mo.Actor): _session_api: Optional[SessionAPI] _storage_api: Optional[StorageAPI] _meta_api: Optional[MetaAPI] - _processor: Optional[SubtaskProcessor] _last_processor: Optional[SubtaskProcessor] _running_aio_task: Optional[asyncio.Task] @@ -538,9 +558,8 @@ def __init__( self._subtask_processor_cls = subtask_processor_cls # current processor - self._processor = None self._last_processor = None - self._running_aio_task = None + self._running_aio_tasks = set() self._session_api = None self._storage_api = None @@ -575,7 +594,12 @@ async def _init_context(self, session_id: str): await context.init() set_context(context) - async def run(self, subtask: Subtask): + async def run( + self, + subtask: Subtask, + forward_successors: bool = False, + finish_callback: Optional[ActorCallback] = None, + ): logger.debug("Start to run subtask: %s", subtask.subtask_id) assert subtask.session_id == self._session_id @@ -589,30 +613,33 @@ async def run(self, subtask: Subtask): self._meta_api, self._band, self._supervisor_address, + forward_successors=forward_successors, + finish_callback=finish_callback, ) - self._processor = self._last_processor = processor - self._running_aio_task = asyncio.create_task(processor.run()) + self._last_processor = processor + aio_task = asyncio.create_task(processor.run()) + self._running_aio_tasks.add(aio_task) try: - result = yield self._running_aio_task + result = yield aio_task raise mo.Return(result) finally: - self._processor = self._running_aio_task = None + self._running_aio_tasks.remove(aio_task) async def wait(self): - return self._processor.is_done.wait() + return self._last_processor.is_done.wait() async def result(self): return self._last_processor.result async def cancel(self): - logger.debug("Cancelling subtask: %s", self._processor.subtask_id) + logger.debug("Cancelling subtask: %s", self._last_processor.subtask_id) - aio_task = self._running_aio_task - aio_task.cancel() + for aio_task in self._running_aio_tasks: + aio_task.cancel() async def waiter(): try: - await aio_task + await asyncio.wait(self._running_aio_tasks) except asyncio.CancelledError: pass @@ -620,7 +647,9 @@ async def waiter(): return waiter() def get_running_subtask_id(self): - return self._processor.subtask_id + if self._last_processor.is_released: + return None + return self._last_processor.subtask_id def set_running_op_progress(self, op_key: str, progress: float): - self._processor.set_op_progress(op_key, progress) + self._last_processor.set_op_progress(op_key, progress) diff --git a/mars/services/subtask/worker/runner.py b/mars/services/subtask/worker/runner.py index 24b6a31d25..7f9a7cc16a 100644 --- a/mars/services/subtask/worker/runner.py +++ b/mars/services/subtask/worker/runner.py @@ -21,6 +21,7 @@ from ....lib.aio import alru_cache from ....typing import BandType from ...cluster import ClusterAPI +from ...core import ActorCallback from ..core import Subtask, SubtaskResult from ..errors import SlotOccupiedAlready from .processor import SubtaskProcessor, SubtaskProcessorActor @@ -82,14 +83,20 @@ async def _get_supervisor_address(self, session_id: str): [address] = await self._cluster_api.get_supervisors_by_keys([session_id]) return address - async def run_subtask(self, subtask: Subtask): + async def run_subtask( + self, + subtask: Subtask, + forward_successors: bool = False, + finish_callback: Optional[ActorCallback] = None, + ): if self._running_processor is not None: # pragma: no cover running_subtask_id = await self._running_processor.get_running_subtask_id() - # current subtask is still running - raise SlotOccupiedAlready( - f"There is subtask(id: {running_subtask_id}) running in {self.uid} " - f"at {self.address}, cannot run subtask {subtask.subtask_id}" - ) + if running_subtask_id is not None: + # current subtask is still running + raise SlotOccupiedAlready( + f"There is subtask(id: {running_subtask_id}) running in {self.uid} " + f"at {self.address}, cannot run subtask {subtask.subtask_id}" + ) session_id = subtask.session_id supervisor_address = await self._get_supervisor_address(session_id) @@ -115,7 +122,9 @@ async def run_subtask(self, subtask: Subtask): processor = self._session_id_to_processors[session_id] self._running_processor = self._last_processor = processor try: - result = yield self._running_processor.run(subtask) + result = yield self._running_processor.run( + subtask, forward_successors, finish_callback + ) finally: self._running_processor = None raise mo.Return(result) diff --git a/mars/services/task/api/oscar.py b/mars/services/task/api/oscar.py index 4ef8c56f51..7c33334da3 100644 --- a/mars/services/task/api/oscar.py +++ b/mars/services/task/api/oscar.py @@ -99,8 +99,18 @@ async def cancel_task(self, task_id: str): async def get_fetch_tileables(self, task_id: str) -> List[Tileable]: return await self._task_manager_ref.get_task_result_tileables(task_id) + @mo.extensible async def set_subtask_result(self, subtask_result: SubtaskResult): return await self._task_manager_ref.set_subtask_result.tell(subtask_result) + @set_subtask_result.batch + async def batch_set_subtask_result(self, args_list, kwargs_list): + batches = [] + for args, kwargs in zip(args_list, kwargs_list): + batches.append( + self._task_manager_ref.set_subtask_result.delay(*args, **kwargs) + ) + await self._task_manager_ref.set_subtask_result.batch(*batches, send=False) + async def get_last_idle_time(self) -> Union[float, None]: return await self._task_manager_ref.get_last_idle_time() diff --git a/mars/services/task/supervisor/manager.py b/mars/services/task/supervisor/manager.py index 3060973b7b..7cfe019b91 100644 --- a/mars/services/task/supervisor/manager.py +++ b/mars/services/task/supervisor/manager.py @@ -296,6 +296,7 @@ async def get_task_result_tileables(self, task_id: str): return await processor_ref.get_result_tileables() + @mo.extensible async def set_subtask_result(self, subtask_result: SubtaskResult): task_id = subtask_result.task_id try: diff --git a/mars/services/task/supervisor/stage.py b/mars/services/task/supervisor/stage.py index c20bea4826..4afd314f16 100644 --- a/mars/services/task/supervisor/stage.py +++ b/mars/services/task/supervisor/stage.py @@ -66,6 +66,7 @@ def __init__( self.subtask_snapshots: Dict[Subtask, SubtaskResult] = dict() self.subtask_results: Dict[Subtask, SubtaskResult] = dict() self._submitted_subtask_ids = set() + self._cached_subtask_ids = set() # All subtask IDs whose input chunk reference count is reduced. self.decref_subtask = set() @@ -90,10 +91,26 @@ def is_cancelled(self): async def _schedule_subtasks(self, subtasks: List[Subtask]): if not subtasks: return + to_cache_subtasks = [] + for subtask in subtasks: + for succ_subtask in self.subtask_graph.successors(subtask): + if ( + succ_subtask.subtask_id in self._submitted_subtask_ids + or succ_subtask.subtask_id in self._cached_subtask_ids + ): + continue + to_cache_subtasks.append(succ_subtask) + self._cached_subtask_ids.update( + subtask.subtask_id for subtask in to_cache_subtasks + ) self._submitted_subtask_ids.update(subtask.subtask_id for subtask in subtasks) - return await self._scheduling_api.add_subtasks( + await self._scheduling_api.add_subtasks( subtasks, [subtask.priority for subtask in subtasks] ) + if to_cache_subtasks: + await self._scheduling_api.cache_subtasks( + to_cache_subtasks, [subtask.priority for subtask in to_cache_subtasks] + ) async def _update_chunks_meta(self, chunk_graph: ChunkGraph): get_meta = [] diff --git a/mars/services/tests/fault_injection_patch.py b/mars/services/tests/fault_injection_patch.py index 1e229c8677..6c410a4278 100644 --- a/mars/services/tests/fault_injection_patch.py +++ b/mars/services/tests/fault_injection_patch.py @@ -19,7 +19,7 @@ from ...lib.aio import alru_cache from ...tests.core import patch_cls, patch_super as super from ..session import SessionAPI -from ..scheduling.worker.execution import SubtaskExecutionActor +from ..scheduling.worker.exec import SubtaskExecutionActor from ..subtask import Subtask from ..subtask.worker.processor import SubtaskProcessor from ..tests.fault_injection_manager import ( diff --git a/mars/utils.py b/mars/utils.py index a933278b97..998d59b79b 100644 --- a/mars/utils.py +++ b/mars/utils.py @@ -1262,14 +1262,12 @@ def dataslots(cls): # Need to create a new class, since we can't set __slots__ # after a class has been created. - # Make sure __slots__ isn't already set. - if "__slots__" in cls.__dict__: # pragma: no cover - raise TypeError(f"{cls.__name__} already specifies __slots__") + slots = getattr(cls, "__slots__", None) or () # Create a new dict for our new class. cls_dict = dict(cls.__dict__) field_names = tuple(f.name for f in dataclasses.fields(cls)) - cls_dict["__slots__"] = field_names + cls_dict["__slots__"] = slots + field_names for field_name in field_names: # Remove our attributes, if present. They'll still be # available in _MARKER.