Skip to content
Merged
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
1 change: 1 addition & 0 deletions .github/workflows/benchmark-ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ jobs:
asv check -E existing
git remote add upstream https://github.com/mars-project/mars.git
git fetch upstream
git merge upstream/master
asv machine --yes
asv continuous -f 1.1 --strict upstream/master HEAD
if: ${{ steps.build.outcome == 'success' }}
Expand Down
12 changes: 12 additions & 0 deletions docs/source/reference/tensor/special.rst
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,18 @@ Error function
mars.tensor.special.erf


Ellipsoidal harmonics
---------------------

.. autosummary::
:toctree: generated/
:nosignatures:

mars.tensor.special.ellip_harm
mars.tensor.special.ellip_harm_2
mars.tensor.special.ellip_normal


Gamma and related functions
---------------------------

Expand Down
14 changes: 11 additions & 3 deletions mars/services/task/analyzer/analyzer.py
Original file line number Diff line number Diff line change
Expand Up @@ -281,14 +281,18 @@ def _gen_logic_key(self, chunks: List[ChunkType]):
)

@enter_mode(build=True)
def gen_subtask_graph(self) -> SubtaskGraph:
def gen_subtask_graph(
self, op_to_bands: Dict[str, BandType] = None
) -> SubtaskGraph:
"""
Analyze chunk graph and generate subtask graph.

Returns
-------
subtask_graph: SubtaskGraph
Subtask graph.
op_to_bands: Dict
Assigned operand's band, usually for fetch operands.
"""
reassign_worker_ops = [
chunk.op for chunk in self._chunk_graph if chunk.op.reassign_worker
Expand All @@ -310,6 +314,8 @@ def gen_subtask_graph(self) -> SubtaskGraph:
for op in start_ops
if op.expect_worker is not None
}
if op_to_bands:
cur_assigns.update(op_to_bands)
logger.debug(
"Start to assign %s start chunks for task %s",
len(start_ops),
Expand Down Expand Up @@ -360,7 +366,8 @@ def gen_subtask_graph(self) -> SubtaskGraph:
chunk_to_colors[c] = op_to_colors[c.op]
color_to_chunks = defaultdict(list)
for chunk, color in chunk_to_colors.items():
color_to_chunks[color].append(chunk)
if not isinstance(chunk.op, Fetch):
color_to_chunks[color].append(chunk)

# gen subtask graph
subtask_graph = SubtaskGraph()
Expand All @@ -370,7 +377,8 @@ def gen_subtask_graph(self) -> SubtaskGraph:
visited = set()
logic_key_to_subtasks = defaultdict(list)
for chunk in self._chunk_graph.topological_iter():
if chunk in visited:
if chunk in visited or isinstance(chunk.op, Fetch):
# skip fetch chunk
continue

color = chunk_to_colors[chunk]
Expand Down
73 changes: 73 additions & 0 deletions mars/services/task/analyzer/tests/test_assigner.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
# 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 numpy as np

from .....config import Config
from .....core import ChunkGraph
from .....tensor.random import TensorRand
from .....tensor.arithmetic import TensorAdd
from .....tensor.fetch import TensorFetch
from .....resource import Resource
from ...core import Task
from ..analyzer import GraphAnalyzer
from ..assigner import GraphAssigner


def test_assigner_with_fetch_inputs():
band_num = 8
all_bands = [(f"address_{i}", "numa-0") for i in range(band_num)]
inputs = [
TensorFetch(key=str(i), source_key=str(i), dtype=np.dtype(int)).new_chunk([])
for i in range(band_num)
]
no_fetch_inputs = [TensorRand(i).new_chunk([]) for i in range(4)]
results = [TensorAdd(lhs=inp, rhs=1).new_chunk([inp]) for inp in inputs]
cur_assigns = dict(
(fetch_chunk.op.key, band[0][0])
for fetch_chunk, band in zip(reversed(inputs), all_bands)
)

chunk_graph = ChunkGraph()
for fetch_chunk, add_chunk in zip(inputs, results):
chunk_graph.add_node(fetch_chunk)
chunk_graph.add_node(add_chunk)
chunk_graph.add_edge(fetch_chunk, add_chunk)
for inp in no_fetch_inputs:
results.append(inp)
chunk_graph.add_node(inp)
chunk_graph.results = results

band_resource = dict((band, Resource(num_cpus=1)) for band in all_bands)

task = Task("mock_task", "mock_session")
analyzer = GraphAnalyzer(chunk_graph, band_resource, task, Config())
subtask_graph = analyzer.gen_subtask_graph(cur_assigns)

assigner = GraphAssigner(
chunk_graph, list(GraphAnalyzer._iter_start_ops(chunk_graph)), band_resource
)
assigns = assigner.assign(cur_assigns)
key_to_assign = dict((c.key, band) for c, band in assigns.items())
for subtask in subtask_graph:
input_chunks = list(subtask.chunk_graph.iter_indep())
if all(isinstance(inp.op, TensorFetch) for inp in input_chunks):
# all inputs are fetch, expect band should be None
assert subtask.expect_band is None
else:
# if subtask has truly initial chunks, expect band should be
# same as assign results
for inp in input_chunks:
if not isinstance(inp.op, TensorFetch):
assert subtask.expect_band == key_to_assign[inp.key]
8 changes: 4 additions & 4 deletions mars/services/task/api/web.py
Original file line number Diff line number Diff line change
Expand Up @@ -247,11 +247,11 @@ async def get_last_idle_time(self) -> Union[float, None]:

async def wait_task(self, task_id: str, timeout: float = None):
path = f"{self._address}/api/session/{self._session_id}/task/{task_id}"
# client timeout should be longer than server timeout.
server_timeout = "" if timeout is None else str(max(timeout / 2.0, timeout - 1))
params = {"action": "wait", "timeout": server_timeout}
# increase client timeout to handle network overhead during entire request
client_timeout = timeout + 3 if timeout else 0
params = {"action": "wait", "timeout": "" if timeout is None else str(timeout)}
res = await self._request_url(
"GET", path, params=params, request_timeout=timeout or 0
"GET", path, params=params, request_timeout=client_timeout
)
return _json_deserial_task_result(json.loads(res.body.decode()))

Expand Down
4 changes: 2 additions & 2 deletions mars/services/task/execution/api.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,14 @@
from dataclasses import dataclass
from typing import List, Dict, Any, Type

from ....core import ChunkGraph
from ....core import ChunkGraph, Chunk
from ....typing import BandType
from ...subtask import SubtaskGraph, SubtaskResult


@dataclass
class ExecutionChunkResult:
key: str # The chunk key for fetching the result.
chunk: Chunk # The chunk key for fetching the result.
meta: Dict # The chunk meta for iterative tiling.
context: Any # The context info, e.g. ray.ObjectRef.

Expand Down
19 changes: 10 additions & 9 deletions mars/services/task/execution/mars/executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
from typing import Dict, List, Optional

from ..... import oscar as mo
from .....core import ChunkGraph
from .....core import ChunkGraph, TileableGraph
from .....core.operand import (
Fetch,
MapReduceOperand,
Expand All @@ -36,6 +36,7 @@
from ....meta.api import MetaAPI
from ....scheduling import SchedulingAPI
from ....subtask import Subtask, SubtaskResult, SubtaskStatus, SubtaskGraph
from ...core import Task
from ..api import TaskExecutor, register_executor_cls
from .resource import ResourceEvaluator
from .stage import TaskStageProcessor
Expand All @@ -61,14 +62,14 @@ class MarsTaskExecutor(TaskExecutor):

def __init__(
self,
config,
task,
tileable_graph,
tile_context,
cluster_api,
lifecycle_api,
scheduling_api,
meta_api,
config: Dict,
task: Task,
tileable_graph: TileableGraph,
tile_context: Dict[TileableType, TileableType],
cluster_api: ClusterAPI,
lifecycle_api: LifecycleAPI,
scheduling_api: SchedulingAPI,
meta_api: MetaAPI,
):
self._config = config
self._task = task
Expand Down
2 changes: 1 addition & 1 deletion mars/services/task/execution/mars/stage.py
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ async def _get_stage_result(self):
metas = await self._meta_api.get_chunk_meta.batch(*get_meta)
for chunk, meta in zip(chunks, metas):
execution_chunk_results.append(
ExecutionChunkResult(key=chunk.key, meta=meta, context=None)
ExecutionChunkResult(chunk=chunk, meta=meta, context=None)
)
return execution_chunk_results

Expand Down
3 changes: 2 additions & 1 deletion mars/services/task/supervisor/preprocessor.py
Original file line number Diff line number Diff line change
Expand Up @@ -206,13 +206,14 @@ def analyze(
chunk_graph: ChunkGraph,
available_bands: Dict[BandType, Resource],
stage_id: str = None,
op_to_bands: Dict[str, BandType] = None,
) -> SubtaskGraph:
logger.debug("Start to gen subtask graph for task %s", self._task.task_id)
task = self._task
analyzer = GraphAnalyzer(
chunk_graph, available_bands, task, self._config, stage_id=stage_id
)
graph = analyzer.gen_subtask_graph()
graph = analyzer.gen_subtask_graph(op_to_bands)
logger.debug(
"Generated subtask graph of %s subtasks for task %s",
len(graph),
Expand Down
18 changes: 16 additions & 2 deletions mars/services/task/supervisor/processor.py
Original file line number Diff line number Diff line change
Expand Up @@ -182,12 +182,26 @@ async def _process_stage_chunk_graph(
chunk_graph: ChunkGraph,
):
available_bands = await self._executor.get_available_band_slots()
meta_api = self._executor._meta_api
get_meta_tasks = []
fetch_op_keys = []
for c in chunk_graph.iter_indep():
if isinstance(c.op, Fetch):
get_meta_tasks.append(
meta_api.get_chunk_meta.delay(c.key, fields=["bands"])
)
fetch_op_keys.append(c.op.key)
key_to_bands = await meta_api.get_chunk_meta.batch(*get_meta_tasks)
fetch_op_to_bands = dict(
(key, meta["bands"][0]) for key, meta in zip(fetch_op_keys, key_to_bands)
)
with Timer() as timer:
subtask_graph = await asyncio.to_thread(
self._preprocessor.analyze,
chunk_graph,
available_bands,
stage_id=stage_id,
op_to_bands=fetch_op_to_bands,
)
stage_profiler.set(f"gen_subtask_graph({len(subtask_graph)})", timer.duration)
logger.info(
Expand Down Expand Up @@ -239,8 +253,8 @@ async def _update_meta(
):
result_chunks = [c for c in chunk_graph.results if not isinstance(c.op, Fetch)]
chunk_to_band = {
c: r.meta["bands"][0][0]
for c, r in zip(result_chunks, execution_chunk_results)
result.chunk: result.meta["bands"][0][0]
for result in execution_chunk_results
}
update_meta_chunks = set(result_chunks)
update_meta_tileables = dict()
Expand Down
1 change: 1 addition & 0 deletions mars/services/task/supervisor/tests/task_preprocessor.py
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,7 @@ def analyze(
chunk_graph: ChunkGraph,
available_bands: Dict[BandType, Resource],
stage_id: str,
op_to_bands: Dict[str, BandType] = None,
) -> SubtaskGraph:
# record shapes generated in tile
for n in chunk_graph:
Expand Down