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 #48958

Merged
merged 11 commits into from
Dec 2, 2024

Conversation

kevin85421
Copy link
Member

@kevin85421 kevin85421 commented Nov 27, 2024

Why are these changes needed?

Issue

In the Ray codebase, 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 (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 (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 2: incremental is True

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

Checks

  • Test 1

    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())
    image
  • Test 2

    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"))
    image
  • 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: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
@kevin85421 kevin85421 marked this pull request as ready for review November 27, 2024 09:14
@kevin85421
Copy link
Member Author

I don't know the expected behavior for the ray.data logger inside a Ray task. @omatthew98, would you mind sharing the expected behavior and confirming whether "Test 1" aligns with it?

Copy link
Collaborator

@jjyao jjyao left a comment

Choose a reason for hiding this comment

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

LG.

Need to change python/ray/_private/log.py as well since there is old dictConfig call there.

@kevin85421
Copy link
Member Author

Need to change python/ray/_private/log.py as well since there is old dictConfig call there.

log.generate_logging_config() is called inside ray/__init__.py, which is executed before ray/data/__init__.py. I believe the order is guaranteed. If you prefer to minimize the use of dictConfig, it makes sense to rewrite it with explicit function calls.

@jjyao
Copy link
Collaborator

jjyao commented Nov 27, 2024

yea let's rewrite to avoid any surprises in the future.

kevin85421 and others added 3 commits November 27, 2024 18:59
Signed-off-by: kaihsun <kaihsun@anyscale.com>
Co-authored-by: Jiajun Yao <jeromeyjj@gmail.com>
Signed-off-by: Kai-Hsun Chen <kaihsun@apache.org>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
import threading

# TODO (kevin85421): It is not used, but a weird error occurs if it is removed.
Copy link
Member Author

Choose a reason for hiding this comment

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

The upper one is without import logging.config, and the lower one is with import logging.config.

image

Copy link
Collaborator

Choose a reason for hiding this comment

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

Let's spend some time figuring out why.

Copy link
Member Author

Choose a reason for hiding this comment

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

ray_logging/__init__.py contains a function setup_component_logger, which uses logging.handlers.RotatingFileHandler. However, the file doesn't import logging.handlers.

Instead, log.py imports logging.config, which implicitly imports logging.handlers.

When the runtime environment agent calls setup_component_logger, it fails, causing the Raylet to also terminate.

Copy link
Member Author

Choose a reason for hiding this comment

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

Solution: import logging.handlers in ray_logging/__init__.py directly.

import threading

# TODO (kevin85421): It is not used, but a weird error occurs if it is removed.
Copy link
Collaborator

Choose a reason for hiding this comment

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

Let's spend some time figuring out why.

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

new_test_logger = logging.getLogger("ray.test")
assert old_test_logger.getEffectiveLevel() == logging.DEBUG
Copy link
Collaborator

Choose a reason for hiding this comment

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

so with dictConfig, log level will also be cleared not just handlers?

Copy link
Member Author

Choose a reason for hiding this comment

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

Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
@jjyao jjyao added the go add ONLY when ready to merge, run all tests label Nov 27, 2024
Signed-off-by: kaihsun <kaihsun@anyscale.com>
@rynewang rynewang merged commit bf98e0d into ray-project:master Dec 2, 2024
5 checks passed
jecsand838 pushed a commit to jecsand838/ray that referenced this pull request 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 pull request 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>
ujjawal-khare pushed a commit to ujjawal-khare-27/ray that referenced this pull request Dec 17, 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: ujjawal-khare <ujjawal.khare@dream11.com>
roshankathawate pushed a commit to roshankathawate/ray that referenced this pull request Jan 3, 2025
[core] Ray Core / Ray Data logging configuration leads to unexpected behavior (ray-project#48958)

See merge request xlabs/x77-taiga/ray-oss!143
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.

[Core] Ray Core / Ray Data logging configuration leads to unexpected behavior
4 participants