-
Notifications
You must be signed in to change notification settings - Fork 108
Add stage-level throughput stats #5063
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
Merged
ravwojdyla
merged 2 commits into
marin-community:main
from
wmoss:zephyr-throughput-stats
Apr 23, 2026
+161
−12
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,17 +16,23 @@ | |
|
|
||
| import logging | ||
| import os | ||
| import re | ||
| import sys | ||
| import threading | ||
| import time | ||
| import traceback | ||
| from contextlib import suppress | ||
| from typing import Any | ||
| from typing import Any, TypeVar | ||
| from collections.abc import Iterator | ||
|
|
||
| import cloudpickle | ||
| from rigging.filesystem import open_url | ||
|
|
||
| from zephyr import counters | ||
| from zephyr.execution import ( | ||
| CounterSnapshot, | ||
| ZEPHYR_STAGE_BYTES_PROCESSED_KEY, | ||
| ZEPHYR_STAGE_ITEM_COUNT_KEY, | ||
| _shared_data_path, | ||
| _worker_ctx_var, | ||
| _write_stage_output, | ||
|
|
@@ -35,7 +41,6 @@ | |
|
|
||
| logger = logging.getLogger(__name__) | ||
|
|
||
|
|
||
| SUBPROCESS_COUNTER_FLUSH_INTERVAL = 5.0 | ||
| """How often the subprocess flushes its counter snapshot to the counter file. | ||
|
|
||
|
|
@@ -44,6 +49,25 @@ | |
| """ | ||
|
|
||
|
|
||
| T = TypeVar("T") | ||
|
|
||
|
|
||
| class StatisticsGenerator: | ||
| """Wraps a generator and counts and sizes yielded items.""" | ||
|
|
||
| def __init__(self, stage_name: str) -> None: | ||
| self._stage_name = stage_name | ||
|
|
||
| def wrap(self, gen: Iterator[T]) -> Iterator[T]: | ||
| for item in gen: | ||
| counters.increment(ZEPHYR_STAGE_ITEM_COUNT_KEY.format(stage_name=self._stage_name), 1) | ||
| counters.increment( | ||
| ZEPHYR_STAGE_BYTES_PROCESSED_KEY.format(stage_name=self._stage_name), | ||
| sys.getsizeof(item), | ||
| ) | ||
| yield item | ||
|
|
||
|
|
||
| class _SubprocessWorkerContext: | ||
| """Lightweight WorkerContext for subprocess shard execution. | ||
|
|
||
|
|
@@ -105,6 +129,45 @@ def _periodic_counter_writer( | |
| logger.warning("Failed to flush counter file to %s", counter_file, exc_info=True) | ||
|
|
||
|
|
||
| def _periodic_status_logger( | ||
| stop_event: threading.Event, | ||
| ctx: _SubprocessWorkerContext, | ||
| stage_name: str, | ||
| execution_id: str, | ||
| shard_idx: int, | ||
| total_shards: int, | ||
| monotonic_start: float, | ||
| interval: float, | ||
| ) -> None: | ||
| """Log ``item_count`` / ``bytes_processed`` rates on a fixed interval (cf. coordinator ``_log_status``). | ||
|
|
||
| Runs in a dedicated daemon thread so logs are attributed to that thread name. | ||
| Reads ``ctx._counters`` the same way as the counter flusher (shallow copy). | ||
| """ | ||
| item_key = ZEPHYR_STAGE_ITEM_COUNT_KEY.format(stage_name=stage_name) | ||
| byte_key = ZEPHYR_STAGE_BYTES_PROCESSED_KEY.format(stage_name=stage_name) | ||
| while not stop_event.wait(timeout=interval): | ||
| if sys.is_finalizing(): | ||
| return | ||
| items = ctx._counters.get(item_key, 0) | ||
| bytes_processed = ctx._counters.get(byte_key, 0) | ||
| elapsed = time.monotonic() - monotonic_start | ||
| item_rate = items / elapsed | ||
| byte_rate = bytes_processed / elapsed | ||
| logger.info( | ||
| "[%s] [%s] [%s] shard %d/%d; items=%d (%.1f/s), bytes_processed=%.1fMiB (%.1fMiB/s)", | ||
| execution_id, | ||
| stage_name, | ||
| threading.current_thread().name, | ||
| shard_idx, | ||
| total_shards, | ||
| items, | ||
| item_rate, | ||
| bytes_processed / (1024 * 1024), | ||
| byte_rate / (1024 * 1024), | ||
| ) | ||
|
|
||
|
|
||
| def execute_shard(task_file: str, result_file: str) -> None: | ||
| """Entry point for subprocess shard execution. | ||
|
|
||
|
|
@@ -135,6 +198,7 @@ def execute_shard(task_file: str, result_file: str) -> None: | |
| counter_file = f"{result_file}.counters" | ||
| stop_event = threading.Event() | ||
| flusher: threading.Thread | None = None | ||
| status_logger: threading.Thread | None = None | ||
| result_or_error: Any | ||
| ctx: _SubprocessWorkerContext | None = None | ||
| try: | ||
|
|
@@ -144,6 +208,8 @@ def execute_shard(task_file: str, result_file: str) -> None: | |
| ctx = _SubprocessWorkerContext(chunk_prefix, execution_id) | ||
| _worker_ctx_var.set(ctx) | ||
|
|
||
| shard_monotonic_start = time.monotonic() | ||
|
|
||
| flusher = threading.Thread( | ||
| target=_periodic_counter_writer, | ||
| args=(stop_event, ctx, counter_file, SUBPROCESS_COUNTER_FLUSH_INTERVAL), | ||
|
|
@@ -152,19 +218,40 @@ def execute_shard(task_file: str, result_file: str) -> None: | |
| ) | ||
| flusher.start() | ||
|
|
||
| status_logger = threading.Thread( | ||
| target=_periodic_status_logger, | ||
| args=( | ||
| stop_event, | ||
| ctx, | ||
| task.stage_name, | ||
| execution_id, | ||
| task.shard_idx, | ||
| task.total_shards, | ||
| shard_monotonic_start, | ||
| SUBPROCESS_COUNTER_FLUSH_INTERVAL, | ||
| ), | ||
| daemon=True, | ||
| name="zephyr-subprocess-status-logger", | ||
| ) | ||
| status_logger.start() | ||
|
|
||
| stage_ctx = StageContext( | ||
| shard=task.shard, | ||
| shard_idx=task.shard_idx, | ||
| total_shards=task.total_shards, | ||
| aux_shards=task.aux_shards, | ||
| ) | ||
|
|
||
| stage_dir = f"{chunk_prefix}/{execution_id}/{task.stage_name}" | ||
| # Sanitize for use as a path component: replace non-alphanumeric runs with '-' | ||
| output_stage_name = re.sub(r"[^a-zA-Z0-9_.-]+", "-", task.stage_name).strip("-") | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Moved this into the subprocess worker so I had access to the original stage name here. As far as I can tell, the stage name was previously only used here so I can just do this transformation here. |
||
| stage_dir = f"{chunk_prefix}/{execution_id}/{output_stage_name}" | ||
| external_sort_dir = f"{stage_dir}-external-sort/shard-{task.shard_idx:04d}" | ||
| scatter_op = next((op for op in task.operations if isinstance(op, Scatter)), None) | ||
|
|
||
| output_counter = StatisticsGenerator(task.stage_name) | ||
|
|
||
| result_or_error = _write_stage_output( | ||
| run_stage(stage_ctx, task.operations, external_sort_dir=external_sort_dir), | ||
| output_counter.wrap(run_stage(stage_ctx, task.operations, external_sort_dir=external_sort_dir)), | ||
| source_shard=task.shard_idx, | ||
| stage_dir=stage_dir, | ||
| shard_idx=task.shard_idx, | ||
|
|
@@ -184,6 +271,8 @@ def execute_shard(task_file: str, result_file: str) -> None: | |
| stop_event.set() | ||
| if flusher is not None and flusher.is_alive(): | ||
| flusher.join(timeout=2.0) | ||
| if status_logger is not None and status_logger.is_alive(): | ||
| status_logger.join(timeout=2.0) | ||
|
|
||
| with open(result_file, "wb") as f: | ||
| counters_out = dict(ctx._counters) if ctx is not None else {} | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,42 @@ | ||
| # Copyright The Marin Authors | ||
| # SPDX-License-Identifier: Apache-2.0 | ||
|
|
||
| """Tests for zephyr.subprocess_worker.""" | ||
|
|
||
| import cloudpickle | ||
|
|
||
| import zephyr.subprocess_worker as sw | ||
| from zephyr.execution import ( | ||
| ZEPHYR_STAGE_BYTES_PROCESSED_KEY, | ||
| ZEPHYR_STAGE_ITEM_COUNT_KEY, | ||
| ListShard, | ||
| ShardTask, | ||
| ) | ||
| from zephyr.shuffle import MemChunk | ||
|
|
||
|
|
||
| def test_execute_shard_sets_stage_scoped_output_counters(tmp_path): | ||
| """execute_shard emits output counters under the task's stage name.""" | ||
| chunk_prefix = str(tmp_path / "chunks") | ||
| execution_id = "test-exec" | ||
| stage_name = "test" | ||
| task = ShardTask( | ||
| shard_idx=0, | ||
| total_shards=1, | ||
| shard=ListShard(refs=[MemChunk(list(range(10)))]), | ||
| operations=[], | ||
| stage_name=stage_name, | ||
| ) | ||
|
|
||
| task_file = str(tmp_path / "task.pkl") | ||
| result_file = str(tmp_path / "result.pkl") | ||
| with open(task_file, "wb") as f: | ||
| cloudpickle.dump((task, chunk_prefix, execution_id), f) | ||
|
|
||
| sw.execute_shard(task_file, result_file) | ||
|
|
||
| with open(result_file, "rb") as f: | ||
| _result_or_error, counters_out = cloudpickle.load(f) | ||
|
|
||
| assert counters_out[ZEPHYR_STAGE_ITEM_COUNT_KEY.format(stage_name=stage_name)] == 10 | ||
| assert counters_out[ZEPHYR_STAGE_BYTES_PROCESSED_KEY.format(stage_name=stage_name)] > 0 |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder if we should log these also in/per worker subprocess?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I actually had this during testing, but removed it once I had it logged in the controller. Added it back in a separate commit so it's easy to pull back out if we want to.