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

[BACKPORT][Ray] Implement ray task executor progress (#3008) #3065

Merged
merged 2 commits into from
May 23, 2022
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
2 changes: 2 additions & 0 deletions mars/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ def ray_start_regular_shared(request): # pragma: no cover

@pytest.fixture(scope="module")
def ray_start_regular_shared2(request): # pragma: no cover
os.environ["RAY_kill_idle_workers_interval_ms"] = "0"
param = getattr(request, "param", {})
num_cpus = param.get("num_cpus", 64)
total_memory_mb = num_cpus * 2 * 1024**2
Expand All @@ -48,6 +49,7 @@ def ray_start_regular_shared2(request): # pragma: no cover
yield ray.init(num_cpus=num_cpus, job_config=job_config)
finally:
ray.shutdown()
os.environ.pop("RAY_kill_idle_workers_interval_ms", None)


@pytest.fixture
Expand Down
32 changes: 31 additions & 1 deletion mars/deploy/oscar/tests/test_local.py
Original file line number Diff line number Diff line change
Expand Up @@ -569,7 +569,7 @@ def test_no_default_session():


@pytest.mark.asyncio
async def test_session_progress(create_cluster):
async def test_session_set_progress(create_cluster):
session = get_default_async_session()
assert session.address is not None
assert session.session_id is not None
Expand All @@ -596,6 +596,36 @@ def f1(interval: float, count: int):
assert info.progress() == 1


@pytest.mark.asyncio
async def test_session_get_progress(create_cluster):
session = get_default_async_session()
assert session.address is not None
assert session.session_id is not None

raw = np.random.rand(100, 4)
t = mt.tensor(raw, chunk_size=5)

def f1(c):
time.sleep(0.5)
return c

t1 = t.sum()
r = t1.map_chunk(f1)
info = await session.execute(r)

for _ in range(100):
if 0 < info.progress() < 1:
break
await asyncio.sleep(0.1)
else:
raise Exception(f"progress test failed, actual value {info.progress()}.")

await info
assert info.result() is None
assert info.exception() is None
assert info.progress() == 1


@pytest.fixture
def setup_session():
session = new_session(n_cpu=2, use_uvloop=False)
Expand Down
6 changes: 6 additions & 0 deletions mars/deploy/oscar/tests/test_ray_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -116,3 +116,9 @@ async def test_iterative_tiling(ray_start_regular_shared2, create_cluster):
@pytest.mark.parametrize("config", [{"backend": "ray"}])
def test_sync_execute(config):
test_local.test_sync_execute(config)


@require_ray
@pytest.mark.asyncio
async def test_session_get_progress(ray_start_regular_shared2, create_cluster):
await test_local.test_session_get_progress(create_cluster)
32 changes: 30 additions & 2 deletions mars/services/task/execution/ray/executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,12 @@ def __init__(
self._task_context = {}
self._available_band_resources = None

# For progress
self._pre_all_stages_progress = 0.0
self._pre_all_stages_tile_progress = 0
self._cur_stage_tile_progress = 0
self._cur_stage_output_object_refs = []

@classmethod
async def create(
cls,
Expand Down Expand Up @@ -200,7 +206,12 @@ async def execute_subtask_graph(
logger.info("Stage %s start.", stage_id)
context = self._task_context
output_meta_object_refs = []

self._pre_all_stages_tile_progress = (
self._pre_all_stages_tile_progress + self._cur_stage_tile_progress
)
self._cur_stage_tile_progress = (
self._tile_context.get_all_progress() - self._pre_all_stages_tile_progress
)
logger.info("Submitting %s subtasks of stage %s.", len(subtask_graph), stage_id)
result_meta_keys = {
chunk.key
Expand Down Expand Up @@ -229,6 +240,7 @@ async def execute_subtask_graph(
continue
elif output_count == 1:
output_object_refs = [output_object_refs]
self._cur_stage_output_object_refs.extend(output_object_refs)
if output_meta_keys:
meta_object_ref, *output_object_refs = output_object_refs
# TODO(fyrestone): Fetch(not get) meta object here.
Expand Down Expand Up @@ -259,6 +271,10 @@ async def execute_subtask_graph(

logger.info("Waiting for stage %s complete.", stage_id)
ray.wait(output_object_refs, fetch_local=False)
# Just use `self._cur_stage_tile_progress` as current stage progress
# because current stage is finished, its progress is 1.
self._pre_all_stages_progress += self._cur_stage_tile_progress
self._cur_stage_output_object_refs.clear()
logger.info("Stage %s is complete.", stage_id)
return chunk_to_meta

Expand Down Expand Up @@ -303,7 +319,19 @@ async def get_available_band_resources(self) -> Dict[BandType, Resource]:

async def get_progress(self) -> float:
"""Get the execution progress."""
return 1
stage_progress = 0.0
total = len(self._cur_stage_output_object_refs)
if total > 0:
finished_objects, _ = ray.wait(
self._cur_stage_output_object_refs,
num_returns=total,
timeout=0.1,
fetch_local=False,
)
stage_progress = (
len(finished_objects) / total * self._cur_stage_tile_progress
)
return self._pre_all_stages_progress + stage_progress

async def cancel(self):
"""Cancel execution."""
Expand Down