diff --git a/python/ray/data/_internal/logging.py b/python/ray/data/_internal/logging.py index 0a9a5a8f7093..ae0991e4adbe 100644 --- a/python/ray/data/_internal/logging.py +++ b/python/ray/data/_internal/logging.py @@ -11,6 +11,12 @@ os.path.join(os.path.dirname(__file__), "logging.yaml") ) +# Env. variable to specify the encoding of the file logs when using the default config. +RAY_DATA_LOG_ENCODING_ENV_VAR_NAME = "RAY_DATA_LOG_ENCODING" + +# Env. variable to specify the logging config path use defaults if not set +RAY_DATA_LOGGING_CONFIG_ENV_VAR_NAME = "RAY_DATA_LOGGING_CONFIG" + # To facilitate debugging, Ray Data writes debug logs to a file. However, if Ray Data # logs every scheduler loop, logging might impact performance. So, we add a "TRACE" # level where logs aren't written by default. @@ -89,15 +95,43 @@ def _try_create_handler(self): def configure_logging() -> None: """Configure the Python logger named 'ray.data'. - This function loads the configration YAML specified by the "RAY_DATA_LOGGING_CONFIG" - environment variable. If the variable isn't set, this function loads the + This function loads the configration YAML specified by "RAY_DATA_LOGGING_CONFIG" + environment variable. If the variable isn't set, this function loads the default "logging.yaml" file that is adjacent to this module. + + If "RAY_DATA_LOG_ENCODING" is specified as "JSON" we will enable JSON logging mode + if using the default logging config. """ - config_path = os.environ.get("RAY_DATA_LOGGING_CONFIG", DEFAULT_CONFIG_PATH) - with open(config_path) as file: - config = yaml.safe_load(file) + + def _load_logging_config(config_path: str): + with open(config_path) as file: + config = yaml.safe_load(file) + return config + + # Dynamically load env vars + config_path = os.environ.get(RAY_DATA_LOGGING_CONFIG_ENV_VAR_NAME) + log_encoding = os.environ.get(RAY_DATA_LOG_ENCODING_ENV_VAR_NAME) + + if config_path is not None: + config = _load_logging_config(config_path) + else: + config = _load_logging_config(DEFAULT_CONFIG_PATH) + if log_encoding is not None and log_encoding.upper() == "JSON": + for logger in config["loggers"].values(): + logger["handlers"].remove("file") + logger["handlers"].append("file_json") + logging.config.dictConfig(config) + # After configuring logger, warn if RAY_DATA_LOGGING_CONFIG is used with + # RAY_DATA_LOG_ENCODING, because they are not both supported together. + if config_path is not None and log_encoding is not None: + logger = logging.getLogger(__name__) + logger.warning( + "Using `RAY_DATA_LOG_ENCODING` is not supported with " + + "`RAY_DATA_LOGGING_CONFIG`" + ) + def reset_logging() -> None: """Reset the logger named 'ray.data' to its initial state. diff --git a/python/ray/data/_internal/logging.yaml b/python/ray/data/_internal/logging.yaml index f72abf356f6d..170d7c6605d8 100644 --- a/python/ray/data/_internal/logging.yaml +++ b/python/ray/data/_internal/logging.yaml @@ -4,16 +4,25 @@ disable_existing_loggers: False formatters: ray: format: "%(asctime)s\t%(levelname)s %(filename)s:%(lineno)s -- %(message)s" + ray_json: + class: ray._private.ray_logging.formatters.JSONFormatter filters: console_filter: (): ray.data._internal.logging.HiddenRecordFilter + core_context_filter: + (): ray._private.ray_logging.filters.CoreContextFilter handlers: file: class: ray.data._internal.logging.SessionFileHandler formatter: ray filename: ray-data.log + file_json: + class: ray.data._internal.logging.SessionFileHandler + formatter: ray_json + filename: ray-data.log + filters: [core_context_filter] console: class: ray._private.log.PlainRayHandler formatter: ray diff --git a/python/ray/data/tests/test_logging.py b/python/ray/data/tests/test_logging.py index de18e4452384..ace59dbcda92 100644 --- a/python/ray/data/tests/test_logging.py +++ b/python/ray/data/tests/test_logging.py @@ -124,6 +124,49 @@ def test_custom_config(reset_logging, monkeypatch, tmp_path): assert isinstance(logger.handlers[0], logging.StreamHandler) +def test_json_logging_configuration( + capsys, reset_logging, monkeypatch, shutdown_only, propagate_logs +): + monkeypatch.setenv("RAY_DATA_LOG_ENCODING", "JSON") + ray.init() + + configure_logging() + + logger = logging.getLogger("ray.data") + + # Ensure handlers correctly setup + handlers = logger.handlers + assert len(handlers) == 2 + assert sum(handler.name == "file_json" for handler in handlers) == 1 + assert sum(handler.name == "console" for handler in handlers) == 1 + + logger.info("ham") + logger.debug("turkey") + + log_path = os.path.join(get_log_directory(), "ray-data.log") + with open(log_path) as file: + log_contents = file.read() + + # Validate the log is in JSON format (a basic check for JSON) + assert all( + log_line.startswith("{") and log_line.endswith("}") + for log_line in log_contents.splitlines() + ) + + assert '"message": "ham"' in log_contents + assert '"message": "turkey"' in log_contents + + # Validate console logs are in text mode + console_log_output = capsys.readouterr().err + assert not any( + log_line.startswith("{") and log_line.endswith("}") + for log_line in console_log_output.splitlines() + ) + + assert "ham" in console_log_output + assert "turkey" not in console_log_output + + if __name__ == "__main__": import sys