Skip to content

Commit

Permalink
[core] Ray Core / Ray Data logging configuration leads to unexpected …
Browse files Browse the repository at this point in the history
…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>
  • Loading branch information
2 people authored and ujjawal-khare committed Dec 17, 2024
1 parent e53085e commit bf0dafe
Show file tree
Hide file tree
Showing 5 changed files with 76 additions and 112 deletions.
56 changes: 15 additions & 41 deletions python/ray/_private/log.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
import logging
from logging.config import dictConfig
import threading
from typing import Union

Expand Down Expand Up @@ -74,44 +73,19 @@ def generate_logging_config():
return
logger_initialized = True

formatters = {
"plain": {
"format": (
"%(asctime)s\t%(levelname)s %(filename)s:%(lineno)s -- %(message)s"
),
},
}

handlers = {
"default": {
"()": PlainRayHandler,
"formatter": "plain",
}
}

loggers = {
# Default ray logger; any log message that gets propagated here will be
# logged to the console. Disable propagation, as many users will use
# basicConfig to set up a default handler. If so, logs will be
# printed twice unless we prevent propagation here.
"ray": {
"level": "INFO",
"handlers": ["default"],
"propagate": False,
},
# Special handling for ray.rllib: only warning-level messages passed through
# See https://github.com/ray-project/ray/pull/31858 for related PR
"ray.rllib": {
"level": "WARN",
},
}

dictConfig(
{
"version": 1,
"formatters": formatters,
"handlers": handlers,
"loggers": loggers,
"disable_existing_loggers": False,
}
plain_formatter = logging.Formatter(
"%(asctime)s\t%(levelname)s %(filename)s:%(lineno)s -- %(message)s"
)

default_handler = PlainRayHandler()
default_handler.setFormatter(plain_formatter)

ray_logger = logging.getLogger("ray")
ray_logger.setLevel(logging.INFO)
ray_logger.addHandler(default_handler)
ray_logger.propagate = False

# Special handling for ray.rllib: only warning-level messages passed through
# See https://github.com/ray-project/ray/pull/31858 for related PR
rllib_logger = logging.getLogger("ray.rllib")
rllib_logger.setLevel(logging.WARN)
1 change: 1 addition & 0 deletions python/ray/_private/ray_logging/__init__.py
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
import colorama
from dataclasses import dataclass
import logging
import logging.handlers
import os
import re
import sys
Expand Down
6 changes: 3 additions & 3 deletions python/ray/_private/ray_logging/default_impl.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
def get_dict_config_provider():
from ray._private.ray_logging.logging_config import DefaultDictConfigProvider
def get_logging_configurator():
from ray._private.ray_logging.logging_config import DefaultLoggingConfigurator

return DefaultDictConfigProvider()
return DefaultLoggingConfigurator()
90 changes: 33 additions & 57 deletions python/ray/_private/ray_logging/logging_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,68 +12,47 @@
import time


class DictConfigProvider(ABC):
class LoggingConfigurator(ABC):
@abstractmethod
def get_supported_encodings(self) -> Set[str]:
raise NotImplementedError

@abstractmethod
def get_dict_config(self, encoding: str, log_level: str) -> dict:
def configure_logging(self, encoding: str, log_level: str):
raise NotImplementedError


class DefaultDictConfigProvider(DictConfigProvider):
class DefaultLoggingConfigurator(LoggingConfigurator):
def __init__(self):
self._dict_configs = {
"TEXT": lambda log_level: {
"version": 1,
"disable_existing_loggers": False,
"formatters": {
"text": {
"()": (
f"{TextFormatter.__module__}."
f"{TextFormatter.__qualname__}"
),
},
},
"filters": {
"core_context": {
"()": (
f"{CoreContextFilter.__module__}."
f"{CoreContextFilter.__qualname__}"
),
},
},
"handlers": {
"console": {
"level": log_level,
"class": "logging.StreamHandler",
"formatter": "text",
"filters": ["core_context"],
},
},
"root": {
"level": log_level,
"handlers": ["console"],
},
"loggers": {
"ray": {
"level": log_level,
"handlers": ["console"],
"propagate": False,
}
},
}
self._encoding_to_formatter = {
"TEXT": TextFormatter(),
}

def get_supported_encodings(self) -> Set[str]:
return self._dict_configs.keys()
return self._encoding_to_formatter.keys()

def get_dict_config(self, encoding: str, log_level: str) -> dict:
return self._dict_configs[encoding](log_level)
def configure_logging(self, encoding: str, log_level: str):
formatter = self._encoding_to_formatter[encoding]
core_context_filter = CoreContextFilter()
handler = logging.StreamHandler()
handler.setLevel(log_level)
handler.setFormatter(formatter)
handler.addFilter(core_context_filter)

root_logger = logging.getLogger()
root_logger.setLevel(log_level)
root_logger.addHandler(handler)

_dict_config_provider: DictConfigProvider = default_impl.get_dict_config_provider()
ray_logger = logging.getLogger("ray")
ray_logger.setLevel(log_level)
# Remove all existing handlers added by `ray/__init__.py`.
for h in ray_logger.handlers[:]:
ray_logger.removeHandler(h)
ray_logger.addHandler(handler)
ray_logger.propagate = False


_logging_configurator: LoggingConfigurator = default_impl.get_logging_configurator()


@PublicAPI(stability="alpha")
Expand All @@ -84,19 +63,16 @@ class LoggingConfig:
log_level: str = "INFO"

def __post_init__(self):
if self.encoding not in _dict_config_provider.get_supported_encodings():
if self.encoding not in _logging_configurator.get_supported_encodings():
raise ValueError(
f"Invalid encoding type: {self.encoding}. "
"Valid encoding types are: "
f"{list(_dict_config_provider.get_supported_encodings())}"
f"{list(_logging_configurator.get_supported_encodings())}"
)

def _get_dict_config(self) -> dict:
"""Get the logging configuration based on the encoding type.
Returns:
dict: The logging configuration.
"""
return _dict_config_provider.get_dict_config(self.encoding, self.log_level)
def _configure_logging(self):
"""Set up the logging configuration for the current process."""
_logging_configurator.configure_logging(self.encoding, self.log_level)

def _setup_log_record_factory(self):
old_factory = logging.getLogRecordFactory()
Expand All @@ -121,7 +97,7 @@ def record_factory(*args, **kwargs):
def _apply(self):
"""Set up both the LogRecord factory and the logging configuration."""
self._setup_log_record_factory()
logging.config.dictConfig(self._get_dict_config())
self._configure_logging()


LoggingConfig.__doc__ = f"""
Expand Down Expand Up @@ -153,7 +129,7 @@ def f():
Args:
encoding: Encoding type for the logs. The valid values are
{list(_dict_config_provider.get_supported_encodings())}
{list(_logging_configurator.get_supported_encodings())}
log_level: Log level for the logs. Defaults to 'INFO'. You can set
it to 'DEBUG' to receive more detailed debug logs.
""" # noqa: E501
35 changes: 24 additions & 11 deletions python/ray/tests/test_logging_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -207,17 +207,9 @@ def test_record_with_exception(self):
assert s in formatted


class TestLoggingConfig:
def test_log_level(self):
log_level = "DEBUG"
logging_config = LoggingConfig(log_level=log_level)
dict_config = logging_config._get_dict_config()
assert dict_config["handlers"]["console"]["level"] == log_level
assert dict_config["root"]["level"] == log_level

def test_invalid_dict_config(self):
with pytest.raises(ValueError):
LoggingConfig(encoding="INVALID")._get_dict_config()
def test_invalid_encoding():
with pytest.raises(ValueError):
LoggingConfig(encoding="INVALID")


class TestTextModeE2E:
Expand Down Expand Up @@ -477,6 +469,27 @@ def print_message(self):
assert "(MyActor pid=" not in stderr


def test_configure_both_structured_logging_and_lib_logging(shutdown_only):
"""
Configure the `ray.test` logger. Then, configure the `root` and `ray`
loggers in `ray.init()`. Ensure that the `ray.test` logger is not affected.
"""
script = """
import ray
import logging
old_test_logger = logging.getLogger("ray.test")
assert old_test_logger.getEffectiveLevel() != logging.DEBUG
old_test_logger.setLevel(logging.DEBUG)
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
"""
run_string_as_driver(script)


if __name__ == "__main__":
if os.environ.get("PARALLEL_CI"):
sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__]))
Expand Down

0 comments on commit bf0dafe

Please sign in to comment.