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] Disable Ray Data operator progress bars when running in Ray job #46826

Merged
merged 7 commits into from
Jul 30, 2024

Conversation

scottjlee
Copy link
Contributor

@scottjlee scottjlee commented Jul 27, 2024

Why are these changes needed?

When using Ray Data in a Ray job, logs are spammed in an incoherent fashion. This is due to how the ray_tqdm module handles multiple progress bars (or rather, how it does not handle them) -- with each progress bar writing independently and the outputs including \r (normally used to overwrite the progress bar in console in real time), this results in empty spaces/lines/overlapping progress bar outputs.

To clean up the logs from Ray jobs, we disable individual operator progress bars, and only keep the "global" overall progress bar for the Ray Dataset. Existing progress bar behavior outside of Ray jobs is unaffected (e.g. using Ray Data outside Ray jobs in console / Jupyter notebooks) and shows all progress bars.

For the following script:

import ray
import time
import os

def f(batch):
    time.sleep(10)
    return batch

paths = ["s3://anonymous@air-example-data/iris.csv"]
ds = ray.data.read_csv(paths, override_num_blocks=20)

ds = ds.map_batches(f)

start_t = time.time()
batch_count = 0
for batch in ds.iter_batches(batch_size=None):
    batch_count += 1

print(f"All done! {time.time() - start_t} seconds, {batch_count} batches")
Output in console, not in Ray Job:
2024-07-26 19:02:44,883 INFO streaming_executor.py:108 -- Starting execution of Dataset. Full logs are in /tmp/ray/session_2024-07-26_12-21-02_283749_7367/logs/ray-data
2024-07-26 19:02:44,883 INFO streaming_executor.py:109 -- Execution plan of Dataset: InputDataBuffer[Input] -> TaskPoolMapOperator[ReadCSV] -> TaskPoolMapOperator[MapBatches(f)]
✔️  Dataset execution finished in 31.51 seconds: 100%|█████████████████████████████████████████████████████████████| 20/20 [00:31<00:00,  1.58s/ bundle]]
- ReadCSV->SplitBlocks(20): 0 active, 0 queued, [cpu: 0.0, objects: 0.0B]: : 20 bundle [00:31,  1.58s/ bundle]]
- MapBatches(f): 0 active, 0 queued, [cpu: 0.0, objects: 840.0B]: 100%|████████████████████████████████████████████| 20/20 [00:31<00:00,  1.58s/ bundle]
All done! 31.516483783721924 seconds, 20 batches
Output from Ray Job:
2024-07-26 19:03:37,857	INFO streaming_executor.py:108 -- Starting execution of Dataset. Full logs are in /tmp/ray/session_2024-07-26_12-21-02_283749_7367/logs/ray-data
2024-07-26 19:03:37,857	INFO streaming_executor.py:109 -- Execution plan of Dataset: InputDataBuffer[Input] -> TaskPoolMapOperator[ReadCSV] -> TaskPoolMapOperator[MapBatches(f)]

Running 0: 0 bundle [00:00, ? bundle/s]
Running: 2/10 CPU, 0/0 GPU, 512.0MB/1.0GB object_store_memory: : 0 bundle [00:01, ? bundle/s]
Running: 2/10 CPU, 0/0 GPU, 6.5KB/1.0GB object_store_memory: : 0 bundle [00:11, ? bundle/s]  
Running: 9/10 CPU, 0/0 GPU, 7.7KB/1.0GB object_store_memory: : 0 bundle [00:21, ? bundle/s]
Running: 9/10 CPU, 0/0 GPU, 7.7KB/1.0GB object_store_memory:   0%|          | 0/20 [00:21<?, ? bundle/s]
Running: 9/10 CPU, 0/0 GPU, 7.7KB/1.0GB object_store_memory:  15%|█▌        | 3/20 [00:21<01:59,  7.01s/ bundle]
Running: 7/10 CPU, 0/0 GPU, 4.0KB/1.0GB object_store_memory:  15%|█▌        | 3/20 [00:22<01:59,  7.01s/ bundle]
Running: 7/10 CPU, 0/0 GPU, 4.0KB/1.0GB object_store_memory:  65%|██████▌   | 13/20 [00:22<00:09,  1.30s/ bundle]
Running: 6/10 CPU, 0/0 GPU, 3.7KB/1.0GB object_store_memory:  65%|██████▌   | 13/20 [00:31<00:09,  1.30s/ bundle]
✔️  Dataset execution finished in 32.02 seconds:  65%|██████▌   | 13/20 [00:32<00:09,  1.30s/ bundle]            
✔️  Dataset execution finished in 32.02 seconds: 100%|██████████| 20/20 [00:32<00:00,  1.36s/ bundle]
✔️  Dataset execution finished in 32.02 seconds: 100%|██████████| 20/20 [00:32<00:00,  1.60s/ bundle]
All done! 32.02574801445007 seconds, 20 batches

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Signed-off-by: Scott Lee <sjl@anyscale.com>
progress_bar_enabled = (
DataContext.get_current().enable_progress_bars
and (is_all_to_all or verbose_progress)
and not is_ray_job
Copy link
Contributor

Choose a reason for hiding this comment

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

can we instead make the default enable_progress_bars False for ray jobs?
So users can still set it to True in case needed.

Signed-off-by: Scott Lee <sjl@anyscale.com>
"Running",
dag.num_outputs_total(),
unit="bundle",
enabled=True,
Copy link
Contributor Author

Choose a reason for hiding this comment

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

always enables the global progress bar.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
@scottjlee scottjlee requested a review from raulchen July 30, 2024 01:50
Signed-off-by: Hao Chen <chenh1024@gmail.com>
@raulchen raulchen enabled auto-merge (squash) July 30, 2024 19:59
@github-actions github-actions bot added the go add ONLY when ready to merge, run all tests label Jul 30, 2024
@raulchen raulchen merged commit 882daae into ray-project:master Jul 30, 2024
6 of 7 checks passed
bveeramani pushed a commit that referenced this pull request Aug 9, 2024
…47029)

#46826 introduced a bug, where
the info log regarding hiding operator-level progress bars is always
shown, regardless of whether the code was run via a Ray Job or not.

This PR fixes the bug by moving the check for whether the code is run
via a Ray Job or not into the `DataContext.__post_init__()` method, so
that the check is done only after the DataContext singleton is
initialized.

---------

Signed-off-by: Scott Lee <sjl@anyscale.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants