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

[data] Fix stats for streaming_split #36217

Merged
merged 1 commit into from
Jun 8, 2023
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
10 changes: 9 additions & 1 deletion python/ray/data/_internal/iterator/stream_split_iterator.py
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ def gen_blocks() -> Iterator[Tuple[ObjectRef[Block], BlockMetadata]]:

def stats(self) -> str:
"""Implements DataIterator."""
return self._base_dataset.stats()
return ray.get(self._coord_actor.stats.remote())

def schema(self) -> Union[type, "pyarrow.lib.Schema"]:
"""Implements DataIterator."""
Expand Down Expand Up @@ -132,6 +132,7 @@ def __init__(
self._equal = equal
self._locality_hints = locality_hints
self._lock = threading.RLock()
self._executor = None

# Guarded by self._lock.
self._next_bundle: Dict[int, RefBundle] = {}
Expand All @@ -143,6 +144,7 @@ def gen_epochs():
executor = StreamingExecutor(
copy.deepcopy(dataset.context.execution_options)
)
self._executor = executor

def add_split_op(dag):
return OutputSplitter(dag, n, equal, locality_hints)
Expand All @@ -159,6 +161,12 @@ def add_split_op(dag):
self._next_epoch = gen_epochs()
self._output_iterator = None

def stats(self) -> str:
"""Returns stats from the base dataset."""
if self._executor:
return self._executor.get_stats().to_summary().to_string()
scottjlee marked this conversation as resolved.
Show resolved Hide resolved
return self._base_dataset.stats()

def start_epoch(self, split_idx: int) -> str:
"""Called to start an epoch.

Expand Down
24 changes: 24 additions & 0 deletions python/ray/data/tests/test_stats.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,30 @@ def dummy_map_batches(x):
return x


def test_streaming_split_stats(ray_start_regular_shared):
context = DataContext.get_current()
ds = ray.data.range(1000, parallelism=10)
it = ds.map_batches(dummy_map_batches).streaming_split(1)[0]
list(it.iter_batches())
stats = it.stats()
assert (
canonicalize(stats)
== """Stage N ReadRange->MapBatches(dummy_map_batches): N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Peak heap memory usage (MiB): N min, N max, N mean
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used
* Extra metrics: {'obj_store_mem_alloc': N, 'obj_store_mem_freed': N, \
'obj_store_mem_peak': N}

Stage N split(N, equal=False):
* Extra metrics: {'num_output_N': N}
"""
)


def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats):
context = DataContext.get_current()
context.optimize_fuse_stages = True
Expand Down