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 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
Original file line number Diff line number Diff line change
Expand Up @@ -150,6 +150,7 @@ def __init__(
"Aggregate",
input_op,
sub_progress_bar_names=[
SortTaskSpec.SORT_SAMPLE_SUB_PROGRESS_BAR_NAME,
ExchangeTaskSpec.MAP_SUB_PROGRESS_BAR_NAME,
ExchangeTaskSpec.REDUCE_SUB_PROGRESS_BAR_NAME,
],
Expand Down
7 changes: 4 additions & 3 deletions python/ray/data/_internal/planner/aggregate.py
Original file line number Diff line number Diff line change
Expand Up @@ -55,11 +55,12 @@ def fn(
else:
# Use same number of output partitions.
num_outputs = num_mappers
sample_bar = ctx.sub_progress_bar_dict[
SortTaskSpec.SORT_SAMPLE_SUB_PROGRESS_BAR_NAME
]
# Sample boundaries for aggregate key.
boundaries = SortTaskSpec.sample_boundaries(
blocks,
SortKey(key),
num_outputs,
blocks, SortKey(key), num_outputs, sample_bar
)

agg_spec = SortAggregateTaskSpec(
Expand Down
18 changes: 11 additions & 7 deletions python/ray/data/_internal/planner/exchange/sort_task_spec.py
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,10 @@ def reduce(

@staticmethod
def sample_boundaries(
blocks: List[ObjectRef[Block]], sort_key: SortKey, num_reducers: int
blocks: List[ObjectRef[Block]],
sort_key: SortKey,
num_reducers: int,
sample_bar: Optional[ProgressBar] = None,
) -> List[T]:
"""
Return (num_reducers - 1) items in ascending order from the blocks that
Expand All @@ -162,13 +165,14 @@ def sample_boundaries(
sample_results = [
sample_block.remote(block, n_samples, sort_key) for block in blocks
]
sample_bar = ProgressBar(
SortTaskSpec.SORT_SAMPLE_SUB_PROGRESS_BAR_NAME,
len(sample_results),
unit="block",
)
if sample_bar is None:
sample_bar = ProgressBar(
SortTaskSpec.SORT_SAMPLE_SUB_PROGRESS_BAR_NAME,
len(blocks) * n_samples,
unit="rows",
)
# TODO(zhilong): Update sort sample bar before finished.
samples = sample_bar.fetch_until_complete(sample_results)
sample_bar.close()
del sample_results
samples = [s for s in samples if len(s) > 0]
# The dataset is empty
Expand Down
7 changes: 6 additions & 1 deletion python/ray/data/_internal/planner/sort.py
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,12 @@ def fn(

# Sample boundaries for sort key.
if not sort_key.boundaries:
boundaries = SortTaskSpec.sample_boundaries(blocks, sort_key, num_outputs)
sample_bar = ctx.sub_progress_bar_dict[
SortTaskSpec.SORT_SAMPLE_SUB_PROGRESS_BAR_NAME
]
boundaries = SortTaskSpec.sample_boundaries(
blocks, sort_key, num_outputs, sample_bar
)
else:
boundaries = [(b,) for b in sort_key.boundaries]
num_outputs = len(boundaries) + 1
Expand Down
27 changes: 20 additions & 7 deletions python/ray/data/_internal/progress_bar.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,24 @@ def set_progress_bars(enabled: bool) -> bool:
)


def extract_num_rows(result: Any) -> int:
"""Extract the number of rows from a result object.

Args:
result: The result object from which to extract the number of rows.

Returns:
The number of rows, defaulting to 1 if it cannot be determined.
"""
if hasattr(result, "num_rows"):
return result.num_rows
elif hasattr(result, "__len__"):
# For output is DataFrame,i.e. sort_sample
return len(result)
else:
return 1


class ProgressBar:
"""Thin wrapper around tqdm to handle soft imports.

Expand Down Expand Up @@ -136,9 +154,7 @@ 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
num_rows = extract_num_rows(result)
total_rows_processed += num_rows
self.update(total_rows_processed)

Expand Down Expand Up @@ -167,11 +183,8 @@ 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
num_rows = extract_num_rows(result)
total_rows_processed += num_rows
# TODO(zhilong): Change the total to total_row when init progress bar
self.update(total_rows_processed)

with _canceled_threads_lock:
Expand Down