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

[Core] Ray Core / Ray Data logging configuration leads to unexpected behavior #48732

Closed
omatthew98 opened this issue Nov 13, 2024 · 5 comments · Fixed by #48958
Closed

[Core] Ray Core / Ray Data logging configuration leads to unexpected behavior #48732

omatthew98 opened this issue Nov 13, 2024 · 5 comments · Fixed by #48958
Assignees
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core P0 Issues that should be fixed in short order

Comments

@omatthew98
Copy link
Contributor

omatthew98 commented Nov 13, 2024

What happened + What you expected to happen

We use logging.config.dictConfig(config) to configure the ray data logger (here), but this is also how ray core configures the ray logger (here).

For both of these logging configs, we use disable_existing_loggers: False. The behavior for this is described as (logging docs):

disable_existing_loggers - whether any existing non-root loggers are to be disabled. This setting mirrors the parameter of the same name in fileConfig(). If absent, this parameter defaults to True. This value is ignored if incremental is True.

This description makes it seem like these two logging calls are commutative (regardless of ordering they will produce the same result), but that is not exactly how the python logging module works. If we configure the ray module logger then the ray.data module logger, the results are expected and both are configured. If we instead configure the ray.data module then configure the ray module logger, then the ray.data logging configuration is clobbered. This happens because when configuring the parent logger of a module (e.g. ray module logger is the parent logger of the ray.data module logger), the various handlers associated with the child logger are not guaranteed to be preserved.

Our end goal should be a state where the call order of the logging configurations should not affect the logging behavior.

Versions / Dependencies

ray==2.39.0

Reproduction script

import ray
import logging

def report_logger(logger):
    # Collect this logger and its parents
    loggers = []
    current_logger = logger
    while current_logger:
        loggers.append(current_logger)
        if not current_logger.parent or current_logger.parent == current_logger:
            break
        current_logger = current_logger.parent
    
    # Report the configuration of each logger in the hierarchy
    print(f"Logging configuration for '{logger.name}' and its hierarchy:")
    for log in reversed(loggers):  # Start from the root and go down to the given logger
        print(f"\nLogger: {log.name or 'root'} (Level: {logging.getLevelName(log.level)})")
        if log.handlers:
            print("  Handlers:")
            for handler in log.handlers:
                print(f"    - {handler.__class__.__name__} (Level: {logging.getLevelName(handler.level)})")
        else:
            print("  No handlers configured")

print("BEFORE")
report_logger(logging.getLogger("ray.data"))
print()

import ray.data
ray.init(logging_config=ray.LoggingConfig(encoding="JSON", log_level="INFO"))

print("AFTER:")
report_logger(logging.getLogger("ray.data"))

Issue Severity

High: It blocks me from completing my task.

@omatthew98 omatthew98 added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Nov 13, 2024
@jcotant1 jcotant1 added core Issues that should be addressed in Ray Core data Ray Data-related issues labels Nov 13, 2024
@omatthew98
Copy link
Contributor Author

omatthew98 commented Nov 13, 2024

To be clear about the expected behavior some examples / modifications of the repro:

1. Ray Data Import then Ray Core Init [UNEXPECTED BEHAVIOR]

Script:

print("BEFORE")
report_logger(logging.getLogger("ray.data"))
print()

import ray.data
ray.init(logging_config=ray.LoggingConfig(encoding="JSON", log_level="INFO"))

print("AFTER:")
report_logger(logging.getLogger("ray.data"))

Output:

BEFORE
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: WARNING)
  No handlers configured

Logger: ray (Level: INFO)
  Handlers:
    - PlainRayHandler (Level: NOTSET)

Logger: ray.data (Level: NOTSET)
  No handlers configured

AFTER:
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: INFO)
  Handlers:
    - StreamHandler (Level: INFO)

Logger: ray (Level: INFO)
  Handlers:
    - StreamHandler (Level: INFO)

Logger: ray.data (Level: NOTSET)
  No handlers configured

Notes:
Ray Data configuration ignored, only Ray Core initialization respected.

2. Ray Core Init then Ray Data Import [EXPECTED BEHAVIOR]

Script:

print("BEFORE")
report_logger(logging.getLogger("ray.data"))
print()

ray.init(logging_config=ray.LoggingConfig(encoding="JSON", log_level="INFO"))
import ray.data

print("AFTER:")
report_logger(logging.getLogger("ray.data"))

Output:

BEFORE
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: WARNING)
  No handlers configured

Logger: ray (Level: INFO)
  Handlers:
    - PlainRayHandler (Level: NOTSET)

Logger: ray.data (Level: NOTSET)
  No handlers configured

AFTER:
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: INFO)
  Handlers:
    - StreamHandler (Level: INFO)

Logger: ray (Level: INFO)
  Handlers:
    - StreamHandler (Level: INFO)

Logger: ray.data (Level: DEBUG)
  Handlers:
    - SessionFileHandler (Level: NOTSET)
    - PlainRayHandler (Level: INFO)
    - SessionFileHandler (Level: ERROR)

Notes:
Ray Core configuration is done first, Ray Data configuration is done second, both are respected.

3. Only Ray Data Import [EXPECTED BEHAVIOR]

Script:

print("BEFORE")
report_logger(logging.getLogger("ray.data"))
print()

import ray.data

print("AFTER:")
report_logger(logging.getLogger("ray.data"))

Output:

BEFORE
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: WARNING)
  No handlers configured

Logger: ray (Level: INFO)
  Handlers:
    - PlainRayHandler (Level: NOTSET)

Logger: ray.data (Level: NOTSET)
  No handlers configured

AFTER:
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: WARNING)
  No handlers configured

Logger: ray (Level: INFO)
  Handlers:
    - PlainRayHandler (Level: NOTSET)

Logger: ray.data (Level: DEBUG)
  Handlers:
    - SessionFileHandler (Level: NOTSET)
    - PlainRayHandler (Level: INFO)
    - SessionFileHandler (Level: ERROR)

Notes:
Correctly configures the Ray Data logger.

4. Only Ray Core Init [EXPECTED BEHAVIOR]

Script:

print("BEFORE")
report_logger(logging.getLogger("ray.data"))
print()

ray.init(logging_config=ray.LoggingConfig(encoding="JSON", log_level="INFO"))

print("AFTER:")
report_logger(logging.getLogger("ray.data"))

Output:

BEFORE
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: WARNING)
  No handlers configured

Logger: ray (Level: INFO)
  Handlers:
    - PlainRayHandler (Level: NOTSET)

Logger: ray.data (Level: NOTSET)
  No handlers configured

AFTER:
Logging configuration for 'ray.data' and its hierarchy:

Logger: root (Level: INFO)
  Handlers:
    - StreamHandler (Level: INFO)

Logger: ray (Level: INFO)
  Handlers:
    - StreamHandler (Level: INFO)

Logger: ray.data (Level: NOTSET)
  No handlers configured

Notes:
Correctly configures the Ray Core logger.

@omatthew98 omatthew98 added P0 Issues that should be fixed in short order and removed triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Nov 13, 2024
@alexeykudinkin
Copy link
Contributor

@omatthew98 why don't we just do

  1. Take the current config
  2. Merge in provided one
  3. Set the config back

@omatthew98
Copy link
Contributor Author

Update from some more offline discussion with @alexeykudinkin:
For now, ensuring that the import / happens after the initialization (e.g. ensuring that the ray module logger is configured before the ray.data module logger), should be an immediate workaround. We should still try to improve the behavior as this is a footgun that will result in unexpected results and missing logs that will hinder debugging efforts for users. Ideally we would do what was suggested above, but unfortunately python's logging module does not provide a native way to retrieve the existing logging configuration as a dictionary, so we would have to manually inspect and reconstruct the dict before merging the provided config which would be somewhat hacky.

An alternative to this would be to just wrap this configuration at the ray core level by:

  1. Persist configuration as root level field of the core logging module
  2. Create util configureLogging and have it be called from both core and other libraries
  3. Within the configureLogging function, merge the provided dictionary with the existing dictionary configuration and then update the logging config with the merged config.

@hongpeng-guo
Copy link
Contributor

I am trying to understand the structured logging behavior as well. One of my question is
#1. Ray Data Import then Ray Core Init [UNEXPECTED BEHAVIOR] will wipe out the ray.data logger configuration; but why
#2. Ray Core Init then Ray Data Import [EXPECTED BEHAVIOR] maintains the expected logger configurations of both logger?
It seems by going in order #2, we got the correct logger config for all three loggers: i.e., root by ray.init, ray by ray.init, and finally ray.data by import ray data

@gvspraveen gvspraveen added P1 Issue that should be fixed within a few weeks P0 Issues that should be fixed in short order and removed P0 Issues that should be fixed in short order data Ray Data-related issues P1 Issue that should be fixed within a few weeks labels Nov 18, 2024
@gvspraveen gvspraveen changed the title [Core][Data] Ray Core / Ray Data logging configuration leads to unexpected behavior [Core] Ray Core / Ray Data logging configuration leads to unexpected behavior Nov 18, 2024
@omatthew98
Copy link
Contributor Author

I am trying to understand the structured logging behavior as well. One of my question is #1. Ray Data Import then Ray Core Init [UNEXPECTED BEHAVIOR] will wipe out the ray.data logger configuration; but why #2. Ray Core Init then Ray Data Import [EXPECTED BEHAVIOR] maintains the expected logger configurations of both logger? It seems by going in order #2, we got the correct logger config for all three loggers: i.e., root by ray.init, ray by ray.init, and finally ray.data by import ray data

Yeah I think that understanding is correct. From what I have read, configuring the parent logger after the child logger (so the order of #1), the result will be only the parent logger is configured. If you configure the child logger after the parent logger (so the order of #2), then the two loggers will be configured as expected. I think this is is just an implementation detail for python's logging module that doesn't seem particularly well documented.

@jjyao jjyao added the data Ray Data-related issues label Nov 25, 2024
@jjyao jjyao assigned kevin85421 and unassigned ruisearch42 Nov 26, 2024
@jjyao jjyao removed the data Ray Data-related issues label Nov 27, 2024
rynewang pushed a commit that referenced this issue Dec 2, 2024
…behavior (#48958)

### Issue

In the Ray codebase,
[logging.config.dictConfig](https://docs.python.org/3/library/logging.config.html#logging.config.dictConfig)
may be called multiple times. However, we found that if a logger’s child
loggers are set before the logger is set via `dictConfig`, it may cause
issues.

*
[Example1](https://gist.github.com/kevin85421/24849e06c61f221fd95063a4ce81ca8f)
(incremental: False): The logger `Ray.data` loses its original handler
and uses the `Ray` logger’s handler after the Ray logger is set via
`dictConfig`.
  ```
2024-11-27 04:32:06,213 - Ray.data - INFO - This is an INFO log from
Ray.data.
2024-11-27 04:32:06,213 - Ray.data - WARNING - This is a WARNING log
from Ray.data.
  2024-11-27 04:32:06,213 - Ray.data - INFO - Ray data propagate False
  abc Ray - DEBUG - This is a DEBUG log from Ray.
  abc Ray - ERROR - This is an ERROR log from Ray.
  abc Ray.data - INFO - Another INFO log from Ray.data.
  abc Ray.data - INFO - Ray data propagate True
  ``` 
*
[Example2](https://gist.github.com/kevin85421/9cf6ee70ceec42be3de888174d0c8e6a)
(incremental: True): It looks like `Ray.data`’s handlers are removed
after the `Ray` logger is set via `dictConfig`.
  ```
2024-11-27 04:35:25,379 - Ray.data - INFO - This is an INFO log from
Ray.data.
2024-11-27 04:35:25,379 - Ray.data - WARNING - This is a WARNING log
from Ray.data.
  2024-11-27 04:35:25,379 - Ray.data - INFO - Ray data propagate False
  This is an ERROR log from Ray.
2024-11-27 04:35:25,379 - Ray.data - INFO - Another INFO log from
Ray.data.
  2024-11-27 04:35:25,379 - Ray.data - INFO - Ray data propagate False
  ```

* CPython implementation
  * Case 1: `incremental` is `False`
* If an existing logger is also a child logger of a new logger, the
child logger’s handlers will be reset, and its `propagate` attribute
will be set to true.
* In
[Example1](https://gist.github.com/kevin85421/24849e06c61f221fd95063a4ce81ca8f),
`Ray.data` is not only an existing logger but also a child logger of
Ray. Therefore, its handlers will be reset, and propagate will be set to
true.
* See the function for more details:
https://github.com/python/cpython/blob/71ede1142ddad2d31cc966b8fe4a5aff664f4d53/Lib/logging/config.py#L193-L196
 * Case 2: `incremental` is `True`
    * No handlers & filters will be added to the new logger.
* See the function for more details:
https://github.com/python/cpython/blob/71ede1142ddad2d31cc966b8fe4a5aff664f4d53/Lib/logging/config.py#L906-L915


### Solution

Instead of using `dictConfig` to set the root logger and the Ray logger,
call other functions to set the loggers explicitly.


## Related issue number

Closes #48732

<!-- For example: "Closes #1234" -->

## Checks

* Test 1
  ```python
  import ray
  import logging
  import ray.data
  
ray.init(logging_config=ray.LoggingConfig(encoding="TEXT",
log_level="INFO"))
  
  
  root_logger = logging.getLogger()
  root_logger.info("root logger")
  
  ray_logger = logging.getLogger("ray")
  ray_logger.info("ray logger")
  
  ray_data_logger = logging.getLogger("ray.data")
  ray_data_logger.info("ray data logger")
  
  @ray.remote
  def f():
      root_logger = logging.getLogger()
      root_logger.info("root logger")
      ray_data_logger = logging.getLogger("ray.data")
      ray_data_logger.info("ray data logger")
  
  
  ray.get(f.remote())
  ```
<img width="1440" alt="image"
src="https://github.com/user-attachments/assets/e522a257-28c5-4b3c-ad62-c41e4cd61664">


* Test 2
  ```python
  import ray
  import logging
  
  def report_logger(logger):
      # Collect this logger and its parents
      loggers = []
      current_logger = logger
      while current_logger:
          loggers.append(current_logger)
if not current_logger.parent or current_logger.parent == current_logger:
              break
          current_logger = current_logger.parent
  
      # Report the configuration of each logger in the hierarchy
print(f"Logging configuration for '{logger.name}' and its hierarchy:")
for log in reversed(loggers): # Start from the root and go down to the
given logger
print(f"\nLogger: {log.name or 'root'} (Level:
{logging.getLevelName(log.level)})")
          if log.handlers:
              print("  Handlers:")
              for handler in log.handlers:
print(f" - {handler.__class__.__name__} (Level:
{logging.getLevelName(handler.level)})")
          else:
              print("  No handlers configured")
  
  print("BEFORE")
  report_logger(logging.getLogger("ray.data"))
  print()
  
  import ray.data
ray.init(logging_config=ray.LoggingConfig(encoding="TEXT",
log_level="INFO"))
  
  print("AFTER:")
  report_logger(logging.getLogger("ray.data"))
  ```
<img width="1189" alt="image"
src="https://github.com/user-attachments/assets/9129b22a-f436-40ca-9f42-f1ecacf6c515">

Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: Kai-Hsun Chen <kaihsun@apache.org>
Co-authored-by: Jiajun Yao <jeromeyjj@gmail.com>
jecsand838 pushed a commit to jecsand838/ray that referenced this issue Dec 4, 2024
…behavior (ray-project#48958)

### Issue

In the Ray codebase,
[logging.config.dictConfig](https://docs.python.org/3/library/logging.config.html#logging.config.dictConfig)
may be called multiple times. However, we found that if a logger’s child
loggers are set before the logger is set via `dictConfig`, it may cause
issues.

*
[Example1](https://gist.github.com/kevin85421/24849e06c61f221fd95063a4ce81ca8f)
(incremental: False): The logger `Ray.data` loses its original handler
and uses the `Ray` logger’s handler after the Ray logger is set via
`dictConfig`.
  ```
2024-11-27 04:32:06,213 - Ray.data - INFO - This is an INFO log from
Ray.data.
2024-11-27 04:32:06,213 - Ray.data - WARNING - This is a WARNING log
from Ray.data.
  2024-11-27 04:32:06,213 - Ray.data - INFO - Ray data propagate False
  abc Ray - DEBUG - This is a DEBUG log from Ray.
  abc Ray - ERROR - This is an ERROR log from Ray.
  abc Ray.data - INFO - Another INFO log from Ray.data.
  abc Ray.data - INFO - Ray data propagate True
  ```
*
[Example2](https://gist.github.com/kevin85421/9cf6ee70ceec42be3de888174d0c8e6a)
(incremental: True): It looks like `Ray.data`’s handlers are removed
after the `Ray` logger is set via `dictConfig`.
  ```
2024-11-27 04:35:25,379 - Ray.data - INFO - This is an INFO log from
Ray.data.
2024-11-27 04:35:25,379 - Ray.data - WARNING - This is a WARNING log
from Ray.data.
  2024-11-27 04:35:25,379 - Ray.data - INFO - Ray data propagate False
  This is an ERROR log from Ray.
2024-11-27 04:35:25,379 - Ray.data - INFO - Another INFO log from
Ray.data.
  2024-11-27 04:35:25,379 - Ray.data - INFO - Ray data propagate False
  ```

* CPython implementation
  * Case 1: `incremental` is `False`
* If an existing logger is also a child logger of a new logger, the
child logger’s handlers will be reset, and its `propagate` attribute
will be set to true.
* In
[Example1](https://gist.github.com/kevin85421/24849e06c61f221fd95063a4ce81ca8f),
`Ray.data` is not only an existing logger but also a child logger of
Ray. Therefore, its handlers will be reset, and propagate will be set to
true.
* See the function for more details:
https://github.com/python/cpython/blob/71ede1142ddad2d31cc966b8fe4a5aff664f4d53/Lib/logging/config.py#L193-L196
 * Case 2: `incremental` is `True`
    * No handlers & filters will be added to the new logger.
* See the function for more details:
https://github.com/python/cpython/blob/71ede1142ddad2d31cc966b8fe4a5aff664f4d53/Lib/logging/config.py#L906-L915

### Solution

Instead of using `dictConfig` to set the root logger and the Ray logger,
call other functions to set the loggers explicitly.

## Related issue number

Closes ray-project#48732

<!-- For example: "Closes ray-project#1234" -->

## Checks

* Test 1
  ```python
  import ray
  import logging
  import ray.data

ray.init(logging_config=ray.LoggingConfig(encoding="TEXT",
log_level="INFO"))

  root_logger = logging.getLogger()
  root_logger.info("root logger")

  ray_logger = logging.getLogger("ray")
  ray_logger.info("ray logger")

  ray_data_logger = logging.getLogger("ray.data")
  ray_data_logger.info("ray data logger")

  @ray.remote
  def f():
      root_logger = logging.getLogger()
      root_logger.info("root logger")
      ray_data_logger = logging.getLogger("ray.data")
      ray_data_logger.info("ray data logger")

  ray.get(f.remote())
  ```
<img width="1440" alt="image"
src="https://github.com/user-attachments/assets/e522a257-28c5-4b3c-ad62-c41e4cd61664">

* Test 2
  ```python
  import ray
  import logging

  def report_logger(logger):
      # Collect this logger and its parents
      loggers = []
      current_logger = logger
      while current_logger:
          loggers.append(current_logger)
if not current_logger.parent or current_logger.parent == current_logger:
              break
          current_logger = current_logger.parent

      # Report the configuration of each logger in the hierarchy
print(f"Logging configuration for '{logger.name}' and its hierarchy:")
for log in reversed(loggers): # Start from the root and go down to the
given logger
print(f"\nLogger: {log.name or 'root'} (Level:
{logging.getLevelName(log.level)})")
          if log.handlers:
              print("  Handlers:")
              for handler in log.handlers:
print(f" - {handler.__class__.__name__} (Level:
{logging.getLevelName(handler.level)})")
          else:
              print("  No handlers configured")

  print("BEFORE")
  report_logger(logging.getLogger("ray.data"))
  print()

  import ray.data
ray.init(logging_config=ray.LoggingConfig(encoding="TEXT",
log_level="INFO"))

  print("AFTER:")
  report_logger(logging.getLogger("ray.data"))
  ```
<img width="1189" alt="image"
src="https://github.com/user-attachments/assets/9129b22a-f436-40ca-9f42-f1ecacf6c515">

Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: Kai-Hsun Chen <kaihsun@apache.org>
Co-authored-by: Jiajun Yao <jeromeyjj@gmail.com>
Signed-off-by: Connor Sanders <connor@elastiflow.com>
dentiny pushed a commit to dentiny/ray that referenced this issue Dec 7, 2024
…behavior (ray-project#48958)

### Issue

In the Ray codebase,
[logging.config.dictConfig](https://docs.python.org/3/library/logging.config.html#logging.config.dictConfig)
may be called multiple times. However, we found that if a logger’s child
loggers are set before the logger is set via `dictConfig`, it may cause
issues.

*
[Example1](https://gist.github.com/kevin85421/24849e06c61f221fd95063a4ce81ca8f)
(incremental: False): The logger `Ray.data` loses its original handler
and uses the `Ray` logger’s handler after the Ray logger is set via
`dictConfig`.
  ```
2024-11-27 04:32:06,213 - Ray.data - INFO - This is an INFO log from
Ray.data.
2024-11-27 04:32:06,213 - Ray.data - WARNING - This is a WARNING log
from Ray.data.
  2024-11-27 04:32:06,213 - Ray.data - INFO - Ray data propagate False
  abc Ray - DEBUG - This is a DEBUG log from Ray.
  abc Ray - ERROR - This is an ERROR log from Ray.
  abc Ray.data - INFO - Another INFO log from Ray.data.
  abc Ray.data - INFO - Ray data propagate True
  ``` 
*
[Example2](https://gist.github.com/kevin85421/9cf6ee70ceec42be3de888174d0c8e6a)
(incremental: True): It looks like `Ray.data`’s handlers are removed
after the `Ray` logger is set via `dictConfig`.
  ```
2024-11-27 04:35:25,379 - Ray.data - INFO - This is an INFO log from
Ray.data.
2024-11-27 04:35:25,379 - Ray.data - WARNING - This is a WARNING log
from Ray.data.
  2024-11-27 04:35:25,379 - Ray.data - INFO - Ray data propagate False
  This is an ERROR log from Ray.
2024-11-27 04:35:25,379 - Ray.data - INFO - Another INFO log from
Ray.data.
  2024-11-27 04:35:25,379 - Ray.data - INFO - Ray data propagate False
  ```

* CPython implementation
  * Case 1: `incremental` is `False`
* If an existing logger is also a child logger of a new logger, the
child logger’s handlers will be reset, and its `propagate` attribute
will be set to true.
* In
[Example1](https://gist.github.com/kevin85421/24849e06c61f221fd95063a4ce81ca8f),
`Ray.data` is not only an existing logger but also a child logger of
Ray. Therefore, its handlers will be reset, and propagate will be set to
true.
* See the function for more details:
https://github.com/python/cpython/blob/71ede1142ddad2d31cc966b8fe4a5aff664f4d53/Lib/logging/config.py#L193-L196
 * Case 2: `incremental` is `True`
    * No handlers & filters will be added to the new logger.
* See the function for more details:
https://github.com/python/cpython/blob/71ede1142ddad2d31cc966b8fe4a5aff664f4d53/Lib/logging/config.py#L906-L915


### Solution

Instead of using `dictConfig` to set the root logger and the Ray logger,
call other functions to set the loggers explicitly.


## Related issue number

Closes ray-project#48732

<!-- For example: "Closes ray-project#1234" -->

## Checks

* Test 1
  ```python
  import ray
  import logging
  import ray.data
  
ray.init(logging_config=ray.LoggingConfig(encoding="TEXT",
log_level="INFO"))
  
  
  root_logger = logging.getLogger()
  root_logger.info("root logger")
  
  ray_logger = logging.getLogger("ray")
  ray_logger.info("ray logger")
  
  ray_data_logger = logging.getLogger("ray.data")
  ray_data_logger.info("ray data logger")
  
  @ray.remote
  def f():
      root_logger = logging.getLogger()
      root_logger.info("root logger")
      ray_data_logger = logging.getLogger("ray.data")
      ray_data_logger.info("ray data logger")
  
  
  ray.get(f.remote())
  ```
<img width="1440" alt="image"
src="https://github.com/user-attachments/assets/e522a257-28c5-4b3c-ad62-c41e4cd61664">


* Test 2
  ```python
  import ray
  import logging
  
  def report_logger(logger):
      # Collect this logger and its parents
      loggers = []
      current_logger = logger
      while current_logger:
          loggers.append(current_logger)
if not current_logger.parent or current_logger.parent == current_logger:
              break
          current_logger = current_logger.parent
  
      # Report the configuration of each logger in the hierarchy
print(f"Logging configuration for '{logger.name}' and its hierarchy:")
for log in reversed(loggers): # Start from the root and go down to the
given logger
print(f"\nLogger: {log.name or 'root'} (Level:
{logging.getLevelName(log.level)})")
          if log.handlers:
              print("  Handlers:")
              for handler in log.handlers:
print(f" - {handler.__class__.__name__} (Level:
{logging.getLevelName(handler.level)})")
          else:
              print("  No handlers configured")
  
  print("BEFORE")
  report_logger(logging.getLogger("ray.data"))
  print()
  
  import ray.data
ray.init(logging_config=ray.LoggingConfig(encoding="TEXT",
log_level="INFO"))
  
  print("AFTER:")
  report_logger(logging.getLogger("ray.data"))
  ```
<img width="1189" alt="image"
src="https://github.com/user-attachments/assets/9129b22a-f436-40ca-9f42-f1ecacf6c515">

Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: Kai-Hsun Chen <kaihsun@apache.org>
Co-authored-by: Jiajun Yao <jeromeyjj@gmail.com>
Signed-off-by: hjiang <dentinyhao@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core P0 Issues that should be fixed in short order
Projects
None yet
Development

Successfully merging a pull request may close this issue.

8 participants