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] Progress Bar: Sort sample in "rows" and remove the duplicate Sort sample. #47106

Merged
merged 12 commits into from
Aug 15, 2024
Merged
Show file tree
Hide file tree
Changes from 4 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
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
from ray.data._internal.logical.interfaces import LogicalOperator
from ray.data._internal.planner.exchange.interfaces import ExchangeTaskSpec
from ray.data._internal.planner.exchange.shuffle_task_spec import ShuffleTaskSpec
from ray.data._internal.planner.exchange.sort_task_spec import SortKey, SortTaskSpec
from ray.data._internal.planner.exchange.sort_task_spec import SortKey
from ray.data.aggregate import AggregateFn
from ray.data.block import BlockMetadata

Expand Down Expand Up @@ -125,7 +125,6 @@ def __init__(
"Sort",
input_op,
sub_progress_bar_names=[
SortTaskSpec.SORT_SAMPLE_SUB_PROGRESS_BAR_NAME,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for looking into this. I think we actually want to keep this bar here, so that it is initialized as a sub-progress bar for the AbstractAllToAll op, and we should remove the one that is created in SortTaskSpec.sample_boundaries(). We want the "Sort Sample" to be a sub-bar of the overall Sort bar, like this:
357460721-30aa9fc3-8e96-473e-a794-da4fc023093a

In terms of the concrete change, we can update sample_boundaries() to reference the sub-progress bar that is part of the operator. This will also require us to either:

(1) change sample_boundaries from a static method to class method, and use the object's sub-progress bar
(2) add a sub_progress_bar parameter to sample_boundaries(), and at the callers of this method, pass the SortTaskSpec instance's progress bar (e.g. here)

ExchangeTaskSpec.MAP_SUB_PROGRESS_BAR_NAME,
ExchangeTaskSpec.REDUCE_SUB_PROGRESS_BAR_NAME,
],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,8 +164,8 @@ def sample_boundaries(
]
sample_bar = ProgressBar(
SortTaskSpec.SORT_SAMPLE_SUB_PROGRESS_BAR_NAME,
len(sample_results),
unit="block",
total=n_samples * len(blocks),
unit="rows",
)
samples = sample_bar.fetch_until_complete(sample_results)
sample_bar.close()
Expand Down
19 changes: 13 additions & 6 deletions python/ray/data/_internal/progress_bar.py
Original file line number Diff line number Diff line change
Expand Up @@ -136,9 +136,12 @@ def block_until_complete(self, remaining: List[ObjectRef]) -> None:
)
total_rows_processed = 0
for _, result in zip(done, ray.get(done)):
num_rows = (
result.num_rows if hasattr(result, "num_rows") else 1
) # Default to 1 if no row count is available
if hasattr(result, "num_rows"):
num_rows = result.num_rows
elif hasattr(result, "__len__"):
num_rows = len(result)
else:
num_rows = 1
total_rows_processed += num_rows
self.update(total_rows_processed)

Expand Down Expand Up @@ -167,9 +170,13 @@ def fetch_until_complete(self, refs: List[ObjectRef]) -> List[Any]:
total_rows_processed = 0
for ref, result in zip(done, ray.get(done)):
ref_to_result[ref] = result
num_rows = (
result.num_rows if hasattr(result, "num_rows") else 1
) # Default to 1 if no row count is available
if hasattr(result, "num_rows"):
num_rows = result.num_rows
elif hasattr(result, "__len__"):
# For output is DataFrame,i.e. sort_sample
num_rows = len(result)
else:
num_rows = 1
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could you consolidate this logic and the one in block_until_complete() as a static method in ProgressBar or a utility method in the file?

total_rows_processed += num_rows
# TODO(zhilong): Change the total to total_row when init progress bar
self.update(total_rows_processed)
Expand Down