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.