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] Restructure stdout logging #43360

Merged
merged 3 commits into from
Feb 26, 2024
Merged

[Data] Restructure stdout logging #43360

merged 3 commits into from
Feb 26, 2024

Conversation

c21
Copy link
Contributor

@c21 c21 commented Feb 22, 2024

Why are these changes needed?

This PR is to restructure the standard output logging of Ray Data w/ motivation to provide useful information for users, because we heard from multiple users that the logging on standard output is spammy and not that useful.

The change includes:

  • Don't print any info/debug-level log on stdout. The warn/error-level log is still on stdout. All logs are persisted in ray-data.log file.
  • Only print out the file path of ray-data.log file and physical execution plan.
  • Enable verbose progress bar by default, because we find it's generally useful for users and developers to monitor.

The stdout after this PR:

2024-02-22 11:29:41,506	INFO streaming_executor.py:118 -- Starting execution of Dataset. Full log is in /tmp/ray/session_2024-02-22_11-29-22_201527_7239/logs/ray-data.log
2024-02-22 11:29:41,506	INFO streaming_executor.py:119 -- Execution plan of Dataset: InputDataBuffer[Input] -> TaskPoolMapOperator[ReadRange] -> TaskPoolMapOperator[Map(foo)->Map(<lambda>)]

Running: 1.0/10.0 CPU, 0.0/0.0 GPU, 0.0 MiB/512.0 MiB object_store_memory:   0%|                   | 0/10 [00:01<?, ?it/s]
- ReadRange->SplitBlocks(2): 9 active, 0 queued, 0.0 MiB objects:  15%|███                 | 3/20 [00:01<00:08,  2.11it/s]
- Map(foo)->Map(<lambda>): 10 active, 10 queued, 0.0 MiB objects:   0%|                            | 0/10 [00:01<?, ?it/s]

The stdout before this PR:

2024-02-22 11:44:25,607	INFO set_read_parallelism.py:115 -- Using autodetected parallelism=20 for stage ReadRange to satisfy parallelism at least twice the available number of CPUs (10).
2024-02-22 11:44:25,608	INFO set_read_parallelism.py:122 -- To satisfy the requested parallelism of 20, each read task output is split into 2 smaller blocks.
2024-02-22 11:44:25,608	INFO streaming_executor.py:112 -- Executing DAG InputDataBuffer[Input] -> TaskPoolMapOperator[ReadRange] -> TaskPoolMapOperator[Map(foo)->Map(<lambda>)]
2024-02-22 11:44:25,608	INFO streaming_executor.py:113 -- Execution config: ExecutionOptions(resource_limits=ExecutionResources(cpu=None, gpu=None, object_store_memory=None), exclude_resources=ExecutionResources(cpu=0, gpu=0, object_store_memory=0), locality_with_output=False, preserve_order=False, actor_locality_enabled=True, verbose_progress=False)
2024-02-22 11:44:25,608	INFO streaming_executor.py:115 -- Tip: For detailed progress reporting, run `ray.data.DataContext.get_current().execution_options.verbose_progress = True`
Running: 10.0/10.0 CPU, 0.0/0.0 GPU, 0.0 MiB/512.0 MiB object_store_memory:  45%|████▌     | 9/20 [00:02<00:01,  5.56it/s]

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: Cheng Su <scnju13@gmail.com>
Copy link
Member

@bveeramani bveeramani left a comment

Choose a reason for hiding this comment

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

Rather than manually specifying log_to_stdout=False, would it be difficult to configure the logger so that we don't print info and debug levels by default?

Comment on lines 113 to 114
"Starting execution of Dataset. Monitor progress on Ray "
"Dashboard."
Copy link
Member

Choose a reason for hiding this comment

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

I feel like the "Monitor progress on Ray Dashboard" part isn't helpful since you already receive a message when Ray initializes instructing you to view the dashboard.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I was wanted to emphasize that people should use dashboard for monitoring. I can also remove this, WDYT? @raulchen.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed for now.

@c21
Copy link
Contributor Author

c21 commented Feb 22, 2024

Rather than manually specifying log_to_stdout=False, would it be difficult to configure the logger so that we don't print info and debug levels by default?

We need to make sure they are still logging to log file though, and not interfere w/ warn/error-level. @scottjlee do you know if we have an easy way to do that? Thanks.

@scottjlee
Copy link
Contributor

Rather than manually specifying log_to_stdout=False, would it be difficult to configure the logger so that we don't print info and debug levels by default?

We need to make sure they are still logging to log file though, and not interfere w/ warn/error-level. @scottjlee do you know if we have an easy way to do that? Thanks.

I think the easiest way would be to save both stdout_logger and logger in _initialize_logger() as attributes of the DatasetLogger. Then we can modify the stdout_logger's info and debug methods to not do anything, so we do not log to stdout in these cases.

Signed-off-by: Cheng Su <scnju13@gmail.com>
@c21
Copy link
Contributor Author

c21 commented Feb 26, 2024

I think the easiest way would be to save both stdout_logger and logger in _initialize_logger() as attributes of the DatasetLogger. Then we can modify the stdout_logger's info and debug methods to not do anything, so we do not log to stdout in these cases.

Got it, @scottjlee, @bveeramani - how about leaving this as a followup? Two loggers sound complicated to me, and I want us to be able to clear up logging in 2.10.

@@ -251,7 +259,7 @@ def _scheduling_loop_step(self, topology: Topology) -> bool:
"""

if DEBUG_TRACE_SCHEDULING:
logger.get_logger().info("Scheduling loop step...")
logger.get_logger(log_to_stdout=False).info("Scheduling loop step...")
Copy link
Contributor

Choose a reason for hiding this comment

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

this one is already behind a debug flag. can keep log_to_stdout=True for easier debugging

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Make sense, changed.

Signed-off-by: Cheng Su <scnju13@gmail.com>
@c21 c21 merged commit 2bc3bd4 into ray-project:master Feb 26, 2024
8 of 9 checks passed
@c21 c21 deleted the parallelism branch February 26, 2024 21:25
@moinnadeem
Copy link

Hey @bveeramani, @c21, heads up that I found the default of:

Enable verbose progress bar by default, because we find it's generally useful for users and developers to monitor.

Highly, highly annoying. I probably wasted 1.5 hours trying to figure out how to turn it off. There's RAY_DATA_DISABLE_PROGRESS_BARS, but also RAY_DATA_VERBOSE_PROGRESS. Why are there two environmental variables? This is a progress bar too, no?

@bveeramani
Copy link
Member

@moinnadeem sorry to hear that. That sounds super frustrating.

Why are there two environmental variables?

Looks like this is a bug. Created an Issue to track: #44267.

Highly, highly annoying.

Would you mind telling me more? Are you getting spammy behavior where the progress bar doesn't render properly, or is something else?

@moinnadeem
Copy link

Would you mind telling me more? Are you getting spammy behavior where the progress bar doesn't render properly, or is something else?

@bveeramani

The bar spams so much that looking for my print statements is like looking for a needle in a haystack. Things are fine now that I disabled it, but I think printing as much as you do isn't a sane default.

For reference, I probably have 100 Ray outputs for every output in my program! I'm on Slack if you want to chat more, we can set up a time where I can show you this (it contains some sensitive stuff, so can't post on GitHub)

@raulchen
Copy link
Contributor

@moinnadeem I fixed an issue and now the progress bars are supposed to adjust width automatically (so it won't keep printing new lines). One exceptional case is when your python script is launched by a shell script, this will be broken. Does your case happen to be the same?

c21 added a commit that referenced this pull request Apr 17, 2024
This is a followup of #43360 to fix the behavior of disabling progress bar. After this PR, users only need to set `ray.data.DataContext.get_current().enable_progress_bars = False` to disable all progress bar (main + verbose bar per operator).

Tested locally and verified config is working. 

Signed-off-by: Cheng Su <scnju13@gmail.com>
harborn pushed a commit to harborn/ray that referenced this pull request Apr 18, 2024
This is a followup of ray-project#43360 to fix the behavior of disabling progress bar. After this PR, users only need to set `ray.data.DataContext.get_current().enable_progress_bars = False` to disable all progress bar (main + verbose bar per operator).

Tested locally and verified config is working. 

Signed-off-by: Cheng Su <scnju13@gmail.com>
ryanaoleary pushed a commit to ryanaoleary/ray that referenced this pull request Jun 7, 2024
This is a followup of ray-project#43360 to fix the behavior of disabling progress bar. After this PR, users only need to set `ray.data.DataContext.get_current().enable_progress_bars = False` to disable all progress bar (main + verbose bar per operator).

Tested locally and verified config is working. 

Signed-off-by: Cheng Su <scnju13@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

5 participants