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] Add Runtime Metrics String #43790

Merged
merged 2 commits into from
Mar 8, 2024
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
52 changes: 43 additions & 9 deletions python/ray/data/_internal/stats.py
Original file line number Diff line number Diff line change
Expand Up @@ -788,8 +788,17 @@ def to_summary(self) -> "DatasetStatsSummary":
self.global_bytes_spilled,
self.global_bytes_restored,
self.dataset_bytes_spilled,
self.streaming_exec_schedule_s.get(),
)

def runtime_metrics(self) -> str:
"""Generate a string representing the runtime metrics of a Dataset. This is
a high level summary of the time spent in Ray Data code broken down by operator.
It also includes the time spent in the scheduler. Times are shown as the total
time for each operator and percentages of time are shown as a fraction of the
total time for the whole dataset."""
return self.to_summary().runtime_metrics()


@DeveloperAPI
@dataclass
Expand All @@ -805,6 +814,7 @@ class DatasetStatsSummary:
global_bytes_spilled: int
global_bytes_restored: int
dataset_bytes_spilled: int
streaming_exec_schedule_s: float

def to_string(
self,
Expand Down Expand Up @@ -862,7 +872,8 @@ def to_string(
else:
already_printed.add(operator_uuid)
out += str(operators_stats_summary)
if DataContext.get_current().verbose_stats_logs and self.extra_metrics:
verbose_stats_logs = DataContext.get_current().verbose_stats_logs
if verbose_stats_logs and self.extra_metrics:
indent = (
"\t"
if operators_stats_summary and operators_stats_summary.is_sub_operator
Expand All @@ -887,13 +898,7 @@ def to_string(

output_num_rows = self.operators_stats[-1].output_num_rows
total_num_out_rows = output_num_rows["sum"] if output_num_rows else 0
total_wall_time = sum(
[
op_stats.wall_time["sum"]
for op_stats in self.operators_stats
if op_stats.wall_time
]
)
total_wall_time = self.get_total_wall_time()
if total_num_out_rows and self.time_total_s and total_wall_time:
out += "\n"
out += "Dataset throughput:\n"
Expand All @@ -907,9 +912,37 @@ def to_string(
f" {total_num_out_rows / total_wall_time} "
"rows/s\n"
)
if verbose_stats_logs and add_global_stats:
out += "\n" + self.runtime_metrics()

return out

@staticmethod
def _collect_parent_summaries(
curr: "DatasetStatsSummary",
) -> List["DatasetStatsSummary"]:
summs = []
# TODO: Do operators ever have multiple parents? Do we need to deduplicate?
for p in curr.parents:
if p and p.parents:
summs.extend(DatasetStatsSummary._collect_parent_summaries(p))
return summs + [curr]

def runtime_metrics(self) -> str:
def fmt_line(name: str, time: float) -> str:
return f"* {name}: {fmt(time)} ({time / self.time_total_s * 100:.3f}%)\n"

summaries = DatasetStatsSummary._collect_parent_summaries(self)
out = "Runtime Metrics:\n"
for summ in summaries:
op_total_time = sum(
[op_stats.time_total_s for op_stats in summ.operators_stats]
)
out += fmt_line(summ.base_name, op_total_time)
out += fmt_line("Scheduling", self.streaming_exec_schedule_s)
out += fmt_line("Total", self.time_total_s)
return out

def __repr__(self, level=0) -> str:
indent = leveled_indent(level)
operators_stats = "\n".join(
Expand Down Expand Up @@ -946,7 +979,8 @@ def get_total_wall_time(self) -> float:
parent_wall_times = [p.get_total_wall_time() for p in self.parents]
parent_max_wall_time = max(parent_wall_times) if parent_wall_times else 0
return parent_max_wall_time + sum(
ss.wall_time.get("max", 0) for ss in self.operators_stats
ss.wall_time.get("max", 0) if ss.wall_time else 0
for ss in self.operators_stats
)

def get_total_cpu_time(self) -> float:
Expand Down
15 changes: 15 additions & 0 deletions python/ray/data/tests/test_stats.py
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,15 @@ def gen_extra_metrics_str(metrics: str, verbose: bool):
return f"* Extra metrics: {metrics}" + "\n" if verbose else ""


def gen_runtime_metrics_str(op_names: List[str], verbose: bool) -> str:
if not verbose:
return ""
out = "\nRuntime Metrics:\n"
for op in op_names + ["Scheduling", "Total"]:
out += f"* {op}: T (N%)\n"
return out


STANDARD_EXTRA_METRICS = gen_expected_metrics(
is_map=True,
spilled=False,
Expand Down Expand Up @@ -266,6 +275,7 @@ def test_streaming_split_stats(ray_start_regular_shared, restore_data_context):
* In batch formatting: T min, T max, T avg, T total
Streaming split coordinator overhead time: T
"""
f"{gen_runtime_metrics_str(['ReadRange->MapBatches(dummy_map_batches)', 'split(N, equal=False)'], True)}" # noqa: E501
)


Expand Down Expand Up @@ -328,6 +338,7 @@ def test_large_args_scheduling_strategy(
f"Dataset throughput:\n"
f" * Ray Data throughput: N rows/s\n"
f" * Estimated single node throughput: N rows/s\n"
f"{gen_runtime_metrics_str(['ReadRange','MapBatches(dummy_map_batches)'], verbose_stats_logs)}" # noqa: E501
)
assert canonicalize(stats) == expected_stats

Expand Down Expand Up @@ -373,6 +384,7 @@ def test_dataset_stats_basic(
f"Dataset throughput:\n"
f" * Ray Data throughput: N rows/s\n"
f" * Estimated single node throughput: N rows/s\n"
f"{gen_runtime_metrics_str(['ReadRange->MapBatches(dummy_map_batches)'], verbose_stats_logs)}" # noqa: E501
)

ds = ds.map(dummy_map_batches).materialize()
Expand All @@ -397,6 +409,7 @@ def test_dataset_stats_basic(
f"Dataset throughput:\n"
f" * Ray Data throughput: N rows/s\n"
f" * Estimated single node throughput: N rows/s\n"
f"{gen_runtime_metrics_str(['ReadRange->MapBatches(dummy_map_batches)','Map(dummy_map_batches)'], verbose_stats_logs)}" # noqa: E501
)

for batch in ds.iter_batches():
Expand Down Expand Up @@ -448,6 +461,7 @@ def test_dataset_stats_basic(
f"Dataset throughput:\n"
f" * Ray Data throughput: N rows/s\n"
f" * Estimated single node throughput: N rows/s\n"
f"{gen_runtime_metrics_str(['ReadRange->MapBatches(dummy_map_batches)','Map(dummy_map_batches)'], verbose_stats_logs)}" # noqa: E501
)


Expand Down Expand Up @@ -1201,6 +1215,7 @@ def test_spilled_stats(shutdown_only, verbose_stats_logs, restore_data_context):
f"Dataset throughput:\n"
f" * Ray Data throughput: N rows/s\n"
f" * Estimated single node throughput: N rows/s\n"
f"{gen_runtime_metrics_str(['ReadRange->MapBatches(<lambda>)'], verbose_stats_logs)}" # noqa: E501
)

assert canonicalize(ds.stats(), filter_global_stats=False) == expected_stats
Expand Down