Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[WIP][Scheduling] Add worker node failover with lineage #3307

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions mars/deploy/oscar/base_config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ scheduling:
# Max number of concurrent speculative run for a subtask.
max_concurrent_run: 3
subtask_cancel_timeout: 5
failover:
enable_lineage: no
metrics:
backend: console
# If backend is prometheus, then we can add prometheus config as follows:
Expand Down
5 changes: 4 additions & 1 deletion mars/deploy/oscar/ray.py
Original file line number Diff line number Diff line change
Expand Up @@ -632,7 +632,10 @@ async def stop(self):
for worker_address in self._worker_addresses:
await stop_worker(worker_address, self._config)
for pool in self._worker_pools:
await pool.actor_pool.remote("stop")
try:
await pool.actor_pool.remote("stop")
except:
pass
if self._supervisor_pool is not None:
await stop_supervisor(self.supervisor_address, self._config)
await self._supervisor_pool.actor_pool.remote("stop")
Expand Down
8 changes: 8 additions & 0 deletions mars/deploy/oscar/tests/fault_injection_config_with_fo.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
"@inherits": '@default'
third_party_modules:
- mars.services.tests.fault_injection_patch
scheduling:
subtask_max_retries: 2
subtask_max_reschedules: 2
failover:
enable_lineage: yes
28 changes: 1 addition & 27 deletions mars/deploy/oscar/tests/test_fault_injection.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,12 @@
from ....oscar.errors import ServerClosed
from ....remote import spawn
from ....services.tests.fault_injection_manager import (
AbstractFaultInjectionManager,
ExtraConfigKey,
FaultInjectionError,
FaultInjectionUnhandledError,
FaultPosition,
FaultType,
create_fault_injection_manager,
)
from ....tensor.base.psrs import PSRSConcatPivot
from ..local import new_cluster
Expand All @@ -54,32 +54,6 @@ async def fault_cluster(request):
yield client


async def create_fault_injection_manager(
session_id, address, fault_count, fault_type, fault_op_types=None
):
class FaultInjectionManager(AbstractFaultInjectionManager):
def __init__(self):
self._fault_count = fault_count

def set_fault_count(self, count):
self._fault_count = count

def get_fault_count(self):
return self._fault_count

def get_fault(self, pos: FaultPosition, ctx=None) -> FaultType:
# Check op types if fault_op_types provided.
if fault_op_types and type(ctx.get("operand")) not in fault_op_types:
return FaultType.NoFault
if self._fault_count.get(pos, 0) > 0:
self._fault_count[pos] -= 1
return fault_type
return FaultType.NoFault

await FaultInjectionManager.create(session_id, address)
return FaultInjectionManager.name


@pytest.mark.parametrize(
"fault_and_exception",
[
Expand Down
114 changes: 114 additions & 0 deletions mars/deploy/oscar/tests/test_ray_error_recovery.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
# 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 os
import numpy as np
import pandas as pd
import pytest

from ....oscar.errors import ServerClosed
from ....remote import spawn
from ....services.tests.fault_injection_manager import (
FaultType,
FaultPosition,
create_fault_injection_manager,
ExtraConfigKey,
)
from ....tests.core import require_ray
from ....utils import lazy_import

from .... import tensor as mt
from .... import dataframe as md
from ..ray import new_cluster

ray = lazy_import("ray")

CONFIG_FILE = os.path.join(os.path.dirname(__file__), "local_test_with_ray_config.yml")
FAULT_INJECTION_CONFIG_FILE = os.path.join(
os.path.dirname(__file__), "fault_injection_config_with_fo.yml"
)


@pytest.fixture
async def fault_ray_cluster(request):
param = getattr(request, "param", {})
client = await new_cluster(
config=param.get("config", CONFIG_FILE),
worker_num=2,
worker_cpu=2,
)
async with client:
yield client


@pytest.mark.parametrize(
"fault_ray_cluster", [{"config": FAULT_INJECTION_CONFIG_FILE}], indirect=True
)
@pytest.mark.parametrize(
"fault_config",
[
[
FaultType.ProcessExit,
{FaultPosition.ON_RUN_SUBTASK: 1},
pytest.raises(ServerClosed),
["_UnretryableException", "*"],
],
],
)
@require_ray
@pytest.mark.asyncio
async def test_node_failover(fault_ray_cluster, fault_config):
fault_type, fault_count, expect_raises, exception_match = fault_config
name = await create_fault_injection_manager(
session_id=fault_ray_cluster.session.session_id,
address=fault_ray_cluster.session.address,
fault_count=fault_count,
fault_type=fault_type,
)

columns = list("ABCDEFGHIJ")
width = len(columns)

df1 = md.DataFrame(
mt.random.randint(
1,
100,
size=(100, width),
chunk_size=(20, width),
),
columns=columns,
)
df2 = df1.execute()
pd1 = df2.to_pandas()

df3 = df2.rechunk(chunk_size=(10, width))
df4 = df3.execute()

def f(x):
return x + 1

r = spawn(f, args=(1,), retry_when_fail=False)
with expect_raises:
r.execute(extra_config={ExtraConfigKey.FAULT_INJECTION_MANAGER_NAME: name})

df5 = df4.apply(
f,
axis=1,
dtypes=pd.Series([np.dtype(np.int64)] * width, index=columns),
output_type="dataframe",
)
df6 = df5.execute()
pd2 = df6.to_pandas()

pd.testing.assert_frame_equal(pd1 + 1, pd2)
13 changes: 10 additions & 3 deletions mars/oscar/backends/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,8 @@ async def _listen(self, client: Client):
# close failed, ignore it
pass
raise ServerClosed(
f"Remote server {client.dest_address} closed"
f"Remote server {client.dest_address} closed",
address=client.dest_address,
) from None
future = self._client_to_message_futures[client].pop(message.message_id)
future.set_result(message)
Expand Down Expand Up @@ -94,7 +95,10 @@ async def _listen(self, client: Client):

message_futures = self._client_to_message_futures.get(client)
self._client_to_message_futures[client] = dict()
error = ServerClosed(f"Remote server {client.dest_address} closed")
error = ServerClosed(
f"Remote server {client.dest_address} closed",
address=client.dest_address,
)
for future in message_futures.values():
future.set_exception(copy.copy(error))

Expand All @@ -119,7 +123,10 @@ async def call(
except ConnectionError:
# close failed, ignore it
pass
raise ServerClosed(f"Remote server {client.dest_address} closed")
raise ServerClosed(
f"Remote server {client.dest_address} closed",
address=client.dest_address,
)

if not wait:
r = wait_response
Expand Down
3 changes: 2 additions & 1 deletion mars/oscar/backends/ray/communication.py
Original file line number Diff line number Diff line change
Expand Up @@ -505,7 +505,8 @@ async def __on_ray_recv__(self, channel_id: ChannelID, message):
if self.stopped:
raise ServerClosed(
f"Remote server {self.address} closed, but got message {message} "
f"from channel {channel_id}"
f"from channel {channel_id}",
address=self.address,
)
channel = self._channels.get(channel_id)
if not channel:
Expand Down
5 changes: 4 additions & 1 deletion mars/oscar/backends/ray/pool.py
Original file line number Diff line number Diff line change
Expand Up @@ -246,7 +246,10 @@ async def __on_ray_recv__(self, channel_id: ChannelID, message):
"""Method for communication based on ray actors"""
try:
if self._ray_server is None:
raise ServerClosed(f"Remote server {channel_id.dest_address} closed")
raise ServerClosed(
f"Remote server {channel_id.dest_address} closed",
address=channel_id.dest_address,
)
return await self._ray_server.__on_ray_recv__(channel_id, message)
except Exception: # pragma: no cover
return RayChannelException(*sys.exc_info())
Expand Down
14 changes: 14 additions & 0 deletions mars/oscar/errors.py
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,20 @@ class SlotStateError(MarsError):


class ServerClosed(MarsError):
def __init__(self, *args, **kwargs):
self._address = kwargs.pop("address", None)
super().__init__(*args, **kwargs)

@property
def address(self):
return self._address


class DataNotExist(MarsError):
pass


class DuplicatedSubtaskError(MarsError):
pass


Expand Down
16 changes: 11 additions & 5 deletions mars/services/cluster/supervisor/node_info.py
Original file line number Diff line number Diff line change
Expand Up @@ -140,20 +140,26 @@ def get_all_bands(
statuses = statuses or {NodeStatus.READY}
role = role or NodeRole.WORKER
nodes = self._role_to_nodes.get(role, [])
return self.get_bands(nodes, statuses)

def get_bands(
self, addresses: List[str], statuses: Set[NodeStatus] = None
) -> Dict[BandType, Resource]:
statuses = statuses or {NodeStatus.READY}
band_resource = dict()
for node in nodes:
if self._node_infos[node].status not in statuses:
for address in addresses:
if self._node_infos[address].status not in statuses:
continue
node_resource = self._node_infos[node].resource
node_resource = self._node_infos[address].resource
for resource_type, info in node_resource.items():
if resource_type.startswith("numa"):
# cpu
band_resource[(node, resource_type)] = Resource(
band_resource[(address, resource_type)] = Resource(
num_cpus=info["cpu_total"], mem_bytes=info["memory_total"]
)
else: # pragma: no cover
assert resource_type.startswith("gpu")
band_resource[(node, resource_type)] = Resource(
band_resource[(address, resource_type)] = Resource(
num_gpus=info["gpu_total"]
)
return band_resource
Expand Down
18 changes: 18 additions & 0 deletions mars/services/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -295,3 +295,21 @@ def wrap(*args, **kwargs):
return fut.result()

return wrap


class _FailOverContext:
def __init__(self):
self._enable_lineage = False
self.subtask_to_dependency_subtasks = defaultdict(set)

def enable_lineage(self):
self._enable_lineage = True

def is_lineage_enabled(self):
return self._enable_lineage

def cleanup(self):
self.subtask_to_dependency_subtasks.clear()


FailOverContext = _FailOverContext()
3 changes: 2 additions & 1 deletion mars/services/lifecycle/supervisor/tests/test_tracker.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,11 @@
from ..... import oscar as mo
from ..... import tensor as mt
from .....core import tile
from .....oscar.errors import DataNotExist
from ....cluster import MockClusterAPI
from ....meta import MockMetaAPI
from ....session import MockSessionAPI
from ....storage import MockStorageAPI, DataNotExist
from ....storage import MockStorageAPI
from ....task.supervisor.manager import TaskManagerActor
from ... import TileableNotTracked
from ...supervisor.tracker import LifecycleTrackerActor
Expand Down
13 changes: 11 additions & 2 deletions mars/services/scheduling/api/oscar.py
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,10 @@ async def get_subtask_schedule_summaries(
return await self._manager_ref.get_schedule_summaries(task_id)

async def add_subtasks(
self, subtasks: List[Subtask], priorities: Optional[List[Tuple]] = None
self,
subtasks: List[Subtask],
priorities: Optional[List[Tuple]] = None,
schedule_lost_objects: bool = False,
):
"""
Submit subtasks into scheduling service
Expand All @@ -85,7 +88,9 @@ async def add_subtasks(
"""
if priorities is None:
priorities = [subtask.priority or tuple() for subtask in subtasks]
await self._manager_ref.add_subtasks(subtasks, priorities)
await self._manager_ref.add_subtasks(
subtasks, priorities, schedule_lost_objects
)

@mo.extensible
async def update_subtask_priority(self, subtask_id: str, priority: Tuple):
Expand Down Expand Up @@ -155,6 +160,10 @@ async def try_enable_autoscale_in(self):
`disable_autoscale_in` has been invoked."""
await self._autoscaler.try_enable_autoscale_in()

@property
def address(self):
return self._address


class MockSchedulingAPI(SchedulingAPI):
@classmethod
Expand Down
2 changes: 2 additions & 0 deletions mars/services/scheduling/supervisor/assigner.py
Original file line number Diff line number Diff line change
Expand Up @@ -216,6 +216,8 @@ async def assign_subtasks(
async def reassign_subtasks(
self, band_to_queued_num: Dict[BandType, int]
) -> Dict[BandType, int]:
# Note: bands may change
self._update_bands(list(await self._cluster_api.get_all_bands(NodeRole.WORKER)))
move_queued_subtasks = {}
for is_gpu in (False, True):
band_name_prefix = "numa" if not is_gpu else "gpu"
Expand Down
Loading