Skip to content

Commit 81755fb

Browse files
authored
add additional_log_standard_attrs to serve logging config (#51144)
### Changes Introduced 1. **Consistent Logging Configuration** - Added `additional_log_standard_attrs` to Serve's `LoggingConfig`, aligning it with Ray's logging configuration. 2. **Refactored `ServeFormatter`** - `ServeFormatter` now inherits from Ray’s `TextFormatter`. 3. **Fixed Potential Bug in `_proto_to_dict`** - Previously, `_proto_to_dict` retained the data type for repeated fields as `google.protobuf.internal.containers.RepeatedScalarFieldContainer` instead of converting it to a Python `list`. - This caused validation failures in Pydantic. - The issue did not surface earlier because `DeploymentConfig.user_configured_option_names` (the only prior repeated attribute user) explicitly disabled data type validation. 4. schema change to `src/ray/protobuf/serve.proto` ### Discussion & Feedback Needed - If we don’t modify `_proto_to_dict`, alternative approaches include: 1. Disabling type checking for Serve’s `LoggingConfig`. 2. Performing type casting in `from_proto`. - Seeking input from reviewers on the preferred approach. ### How do i know this works 1. updated unit tests 2. Manual test sample code ```python import logging from ray import serve import requests import starlette logger = logging.getLogger("ray.serve") logging_config = { "additional_log_standard_attrs": ["name"], } @serve.deployment(logging_config=logging_config) class Model: def __call__(self, req: starlette.requests.Request): logger.info("This is a test log") return 1 serve.run(Model.bind()) resp = requests.get("http://127.0.0.1:8000/") assert resp.status_code == 200 resp = resp.json() ``` results from changes in this PR ```zsh (myenv) ➜ serve_apps python app1.py 2025-03-07 15:07:18,712 INFO worker.py:1839 -- Started a local Ray instance. View the dashboard at 127.0.0.1:8265 (ProxyActor pid=85544) INFO 2025-03-07 15:07:20,223 proxy 127.0.0.1 -- Proxy starting on node 799527b99924c6f2084364a36726a6874d28bfba171c5a06d619bdb2 (HTTP port: 8000). INFO 2025-03-07 15:07:20,301 serve 85513 -- Started Serve in namespace "serve". (ProxyActor pid=85544) INFO 2025-03-07 15:07:20,252 proxy 127.0.0.1 -- Got updated endpoints: {}. (ServeController pid=85545) INFO 2025-03-07 15:07:20,403 controller 85545 -- Deploying new version of Deployment(name='Model', app='default') (initial target replicas: 1). (ProxyActor pid=85544) INFO 2025-03-07 15:07:20,405 proxy 127.0.0.1 -- Got updated endpoints: {Deployment(name='Model', app='default'): EndpointInfo(route='/', app_is_cross_language=False)}. (ProxyActor pid=85544) INFO 2025-03-07 15:07:20,408 proxy 127.0.0.1 -- Started <ray.serve._private.router.SharedRouterLongPollClient object at 0x12c068b80>. (ServeController pid=85545) INFO 2025-03-07 15:07:20,505 controller 85545 -- Adding 1 replica to Deployment(name='Model', app='default'). INFO 2025-03-07 15:07:21,417 serve 85513 -- Application 'default' is ready at http://127.0.0.1:8000/. (ServeReplica:default:Model pid=85549) /Users/abrar/pg/ray/python/ray/serve/_private/replica.py:1320: UserWarning: Calling sync method '__call__' directly on the asyncio loop. In a future version, sync methods will be run in a threadpool by default. Ensure your sync methods are thread safe or keep the existing behavior by making them `async def`. Opt into the new behavior by setting RAY_SERVE_RUN_SYNC_IN_THREADPOOL=1. (ServeReplica:default:Model pid=85549) warnings.warn( (ServeReplica:default:Model pid=85549) INFO 2025-03-07 15:07:21,434 default_Model pd962afl 4e386730-546d-4693-abb4-72e107cfe87f ray.serve -- This is a test log (ServeReplica:default:Model pid=85549) INFO 2025-03-07 15:07:21,435 default_Model pd962afl 4e386730-546d-4693-abb4-72e107cfe87f ray.serve -- GET / 200 4.1ms ``` results from code from master ```zsh (myenv) ➜ serve_apps python app1.py 2025-03-07 10:01:31,875 INFO worker.py:1839 -- Started a local Ray instance. View the dashboard at 127.0.0.1:8265 (ProxyActor pid=41050) INFO 2025-03-07 10:01:33,414 proxy 127.0.0.1 -- Proxy starting on node 2194f00eb1063128758f6d7e6d5a56f7425845ac6492ca35cf04ab15 (HTTP port: 8000). (ProxyActor pid=41050) INFO 2025-03-07 10:01:33,442 proxy 127.0.0.1 -- Got updated endpoints: {}. INFO 2025-03-07 10:01:33,461 serve 41018 -- Started Serve in namespace "serve". (ServeController pid=41052) INFO 2025-03-07 10:01:33,472 controller 41052 -- Deploying new version of Deployment(name='Model', app='default') (initial target replicas: 1). (ProxyActor pid=41050) INFO 2025-03-07 10:01:33,474 proxy 127.0.0.1 -- Got updated endpoints: {Deployment(name='Model', app='default'): EndpointInfo(route='/', app_is_cross_language=False)}. (ProxyActor pid=41050) INFO 2025-03-07 10:01:33,478 proxy 127.0.0.1 -- Started <ray.serve._private.router.SharedRouterLongPollClient object at 0x1289f14f0>. (ServeController pid=41052) INFO 2025-03-07 10:01:33,575 controller 41052 -- Adding 1 replica to Deployment(name='Model', app='default'). INFO 2025-03-07 10:01:34,574 serve 41018 -- Application 'default' is ready at http://127.0.0.1:8000/. (ServeReplica:default:Model pid=41056) /Users/abrar/pg/ray/python/ray/serve/_private/replica.py:1320: UserWarning: Calling sync method '__call__' directly on the asyncio loop. In a future version, sync methods will be run in a threadpool by default. Ensure your sync methods are thread safe or keep the existing behavior by making them `async def`. Opt into the new behavior by setting RAY_SERVE_RUN_SYNC_IN_THREADPOOL=1. (ServeReplica:default:Model pid=41056) warnings.warn( (ServeReplica:default:Model pid=41056) INFO 2025-03-07 10:01:34,582 default_Model 8e8rohb4 72547bda-8287-4767-910a-0763dbabf96b -- This is a test log (ServeReplica:default:Model pid=41056) INFO 2025-03-07 10:01:34,582 default_Model 8e8rohb4 72547bda-8287-4767-910a-0763dbabf96b -- GET / 200 3.0ms ``` --------- Signed-off-by: Abrar Sheikh <[email protected]>
1 parent 65514ea commit 81755fb

File tree

8 files changed

+122
-14
lines changed

8 files changed

+122
-14
lines changed

python/ray/_private/ray_logging/formatters.py

+4
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,10 @@ def set_additional_log_standard_attrs(
4141
) -> None:
4242
self._additional_log_standard_attrs = additional_log_standard_attrs
4343

44+
@property
45+
def additional_log_standard_attrs(self) -> List[str]:
46+
return self._additional_log_standard_attrs
47+
4448
def generate_record_format_attrs(
4549
self,
4650
record: logging.LogRecord,

python/ray/serve/_private/config.py

+12-3
Original file line numberDiff line numberDiff line change
@@ -60,8 +60,19 @@ def _proto_to_dict(proto: Message) -> Dict:
6060
data = {}
6161
# Fill data with non-empty fields.
6262
for field, value in proto.ListFields():
63+
# Handle repeated fields
64+
if field.label == FieldDescriptor.LABEL_REPEATED:
65+
# if we dont do this block the repeated field will be a list of
66+
# `google.protobuf.internal.containers.RepeatedScalarFieldContainer
67+
# Explicitly convert to list
68+
if field.type == FieldDescriptor.TYPE_MESSAGE:
69+
data[field.name] = [
70+
_proto_to_dict(v) for v in value
71+
] # Convert each item
72+
else:
73+
data[field.name] = list(value) # Convert to list directly
6374
# Recursively call if the field is another protobuf.
64-
if field.type == FieldDescriptor.TYPE_MESSAGE:
75+
elif field.type == FieldDescriptor.TYPE_MESSAGE:
6576
data[field.name] = _proto_to_dict(value)
6677
else:
6778
data[field.name] = value
@@ -74,7 +85,6 @@ def _proto_to_dict(proto: Message) -> Dict:
7485
and not field.containing_oneof # skip optional fields
7586
):
7687
data[field.name] = field.default_value
77-
7888
return data
7989

8090

@@ -198,7 +208,6 @@ def logging_config_valid(cls, v):
198208
from ray.serve.schema import LoggingConfig
199209

200210
v = LoggingConfig(**v).dict()
201-
202211
return v
203212

204213
@validator("max_queued_requests", always=True)

python/ray/serve/_private/logging_utils.py

+23-7
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,7 @@
88
import ray
99
from ray._private.ray_constants import LOGGING_ROTATE_BACKUP_COUNT, LOGGING_ROTATE_BYTES
1010
from ray._private.ray_logging.filters import CoreContextFilter
11-
from ray._private.ray_logging.formatters import JSONFormatter
11+
from ray._private.ray_logging.formatters import JSONFormatter, TextFormatter
1212
from ray.serve._private.common import ServeComponentType
1313
from ray.serve._private.constants import (
1414
RAY_SERVE_ENABLE_CPU_PROFILING,
@@ -111,7 +111,7 @@ def filter(self, record):
111111
return True
112112

113113

114-
class ServeFormatter(logging.Formatter):
114+
class ServeFormatter(TextFormatter):
115115
"""Serve Logging Formatter
116116
117117
The formatter will generate the log format on the fly based on the field of record.
@@ -123,7 +123,12 @@ def __init__(
123123
self,
124124
component_name: str,
125125
component_id: str,
126+
fmt: Optional[str] = None,
127+
datefmt: Optional[str] = None,
128+
style: str = "%",
129+
validate: bool = True,
126130
):
131+
super().__init__(fmt, datefmt, style, validate)
127132
self.component_log_fmt = ServeFormatter.COMPONENT_LOG_FMT.format(
128133
component_name=component_name, component_id=component_id
129134
)
@@ -133,17 +138,18 @@ def format(self, record: logging.LogRecord) -> str:
133138
134139
Args:
135140
record: The log record to be formatted.
136-
137141
Returns:
138142
The formatted log record in string format.
139143
"""
140144
record_format = self.component_log_fmt
141145
record_formats_attrs = []
142146
if SERVE_LOG_REQUEST_ID in record.__dict__:
143147
record_formats_attrs.append(SERVE_LOG_RECORD_FORMAT[SERVE_LOG_REQUEST_ID])
148+
record_formats_attrs.extend(
149+
[f"%({k})s" for k in self.additional_log_standard_attrs]
150+
)
144151
record_formats_attrs.append(SERVE_LOG_RECORD_FORMAT[SERVE_LOG_MESSAGE])
145152
record_format += " ".join(record_formats_attrs)
146-
147153
# create a formatter using the format string
148154
formatter = logging.Formatter(record_format)
149155

@@ -291,11 +297,21 @@ def configure_component_logger(
291297
logger.setLevel(logging_config.log_level)
292298
logger.handlers.clear()
293299

300+
serve_formatter = ServeFormatter(component_name, component_id)
301+
json_formatter = JSONFormatter()
302+
if logging_config.additional_log_standard_attrs:
303+
json_formatter.set_additional_log_standard_attrs(
304+
logging_config.additional_log_standard_attrs
305+
)
306+
serve_formatter.set_additional_log_standard_attrs(
307+
logging_config.additional_log_standard_attrs
308+
)
309+
294310
# Only add stream handler if RAY_SERVE_LOG_TO_STDERR is True or if
295311
# `stream_handler_only` is set to True.
296312
if RAY_SERVE_LOG_TO_STDERR or stream_handler_only:
297313
stream_handler = logging.StreamHandler()
298-
stream_handler.setFormatter(ServeFormatter(component_name, component_id))
314+
stream_handler.setFormatter(serve_formatter)
299315
stream_handler.addFilter(log_to_stderr_filter)
300316
stream_handler.addFilter(ServeContextFilter())
301317
logger.addHandler(stream_handler)
@@ -341,9 +357,9 @@ def configure_component_logger(
341357
file_handler.addFilter(
342358
ServeComponentFilter(component_name, component_id, component_type)
343359
)
344-
file_handler.setFormatter(JSONFormatter())
360+
file_handler.setFormatter(json_formatter)
345361
else:
346-
file_handler.setFormatter(ServeFormatter(component_name, component_id))
362+
file_handler.setFormatter(serve_formatter)
347363

348364
if logging_config.enable_access_log is False:
349365
file_handler.addFilter(log_access_log_filter)

python/ray/serve/schema.py

+20
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515
root_validator,
1616
validator,
1717
)
18+
from ray._private.ray_logging.constants import LOGRECORD_STANDARD_ATTRS
1819
from ray._private.runtime_env.packaging import parse_uri
1920
from ray.serve._private.common import (
2021
DeploymentStatus,
@@ -140,6 +141,15 @@ class Config:
140141
"Whether to enable access logs for each request. Default to True."
141142
),
142143
)
144+
additional_log_standard_attrs: List[str] = Field(
145+
default_factory=list,
146+
description=(
147+
"Default attributes from the Python standard logger that will be "
148+
"added to all log records. "
149+
"See https://docs.python.org/3/library/logging.html#logrecord-attributes "
150+
"for a list of available attributes."
151+
),
152+
)
143153

144154
@validator("encoding")
145155
def valid_encoding_format(cls, v):
@@ -168,6 +178,16 @@ def valid_log_level(cls, v):
168178
)
169179
return v
170180

181+
@validator("additional_log_standard_attrs")
182+
def valid_additional_log_standard_attrs(cls, v):
183+
for attr in v:
184+
if attr not in LOGRECORD_STANDARD_ATTRS:
185+
raise ValueError(
186+
f"Unknown attribute '{attr}'. "
187+
f"Additional log standard attributes must be one of {LOGRECORD_STANDARD_ATTRS}."
188+
)
189+
return list(set(v))
190+
171191
def _compute_hash(self) -> int:
172192
return crc32(
173193
(

python/ray/serve/tests/test_logging.py

+38-1
Original file line numberDiff line numberDiff line change
@@ -597,11 +597,19 @@ def __call__(self, req: starlette.requests.Request):
597597
paths[-1] = "new_dir"
598598
new_log_dir = "/".join(paths)
599599

600-
serve.run(Model.options(logging_config={"logs_dir": new_log_dir}).bind())
600+
serve.run(
601+
Model.options(
602+
logging_config={
603+
"logs_dir": new_log_dir,
604+
"additional_log_standard_attrs": ["name"],
605+
}
606+
).bind()
607+
)
601608
resp = requests.get("http://127.0.0.1:8000/").json()
602609
assert "new_dir" in resp["logs_path"]
603610

604611
check_log_file(resp["logs_path"], [".*model_info_level.*"])
612+
check_log_file(resp["logs_path"], ["ray.serve"], check_contains=True)
605613

606614
@pytest.mark.parametrize("enable_access_log", [True, False])
607615
@pytest.mark.parametrize("encoding_type", ["TEXT", "JSON"])
@@ -636,6 +644,35 @@ def __call__(self, req: starlette.requests.Request):
636644
with pytest.raises(AssertionError):
637645
check_log_file(resp["logs_path"], [".*model_not_show.*"])
638646

647+
@pytest.mark.parametrize("encoding_type", ["TEXT", "JSON"])
648+
def test_additional_log_standard_attrs(self, serve_and_ray_shutdown, encoding_type):
649+
"""Test additional log standard attrs"""
650+
logger = logging.getLogger("ray.serve")
651+
logging_config = {
652+
"enable_access_log": True,
653+
"encoding": encoding_type,
654+
"additional_log_standard_attrs": ["name"],
655+
}
656+
657+
@serve.deployment(logging_config=logging_config)
658+
class Model:
659+
def __call__(self, req: starlette.requests.Request):
660+
logger.info("model_info_level")
661+
logger.info("model_not_show", extra={"serve_access_log": True})
662+
return {
663+
"logs_path": logger.handlers[1].baseFilename,
664+
}
665+
666+
serve.run(Model.bind())
667+
668+
resp = requests.get("http://127.0.0.1:8000/")
669+
assert resp.status_code == 200
670+
resp = resp.json()
671+
if encoding_type == "JSON":
672+
check_log_file(resp["logs_path"], ["name"], check_contains=True)
673+
else:
674+
check_log_file(resp["logs_path"], ["ray.serve"], check_contains=True)
675+
639676
def test_application_logging_overwrite(self, serve_and_ray_shutdown):
640677
@serve.deployment
641678
class Model:

python/ray/serve/tests/unit/test_config.py

+7-3
Original file line numberDiff line numberDiff line change
@@ -769,13 +769,17 @@ def test_nested_protobufs(self):
769769
def test_repeated_field(self):
770770
"""Test _proto_to_dict() to deserialize protobuf with repeated field"""
771771
user_configured_option_names = ["foo", "bar"]
772-
proto = DeploymentConfigProto(
772+
config = DeploymentConfig.from_default(
773773
user_configured_option_names=user_configured_option_names,
774774
)
775+
proto_bytes = config.to_proto_bytes()
776+
proto = DeploymentConfigProto.FromString(proto_bytes)
775777
result = _proto_to_dict(proto)
776-
777778
# Repeated field is filled correctly as list.
778-
assert result["user_configured_option_names"] == user_configured_option_names
779+
assert set(result["user_configured_option_names"]) == set(
780+
user_configured_option_names
781+
)
782+
assert isinstance(result["user_configured_option_names"], list)
779783

780784
def test_enum_field(self):
781785
"""Test _proto_to_dict() to deserialize protobuf with enum field"""

python/ray/serve/tests/unit/test_schema.py

+17
Original file line numberDiff line numberDiff line change
@@ -688,6 +688,7 @@ def test_parse_dict(self):
688688
assert schema.encoding == "JSON"
689689
assert schema.logs_dir == "/my_dir"
690690
assert schema.enable_access_log
691+
assert schema.additional_log_standard_attrs == []
691692

692693
# Test string values for log_level.
693694
schema = LoggingConfig.parse_obj(
@@ -714,6 +715,22 @@ def test_default_values(self):
714715
assert schema.encoding == "TEXT"
715716
assert schema.logs_dir is None
716717
assert schema.enable_access_log
718+
assert schema.additional_log_standard_attrs == []
719+
720+
def test_additional_log_standard_attrs_type(self):
721+
schema = LoggingConfig.parse_obj({"additional_log_standard_attrs": ["name"]})
722+
assert isinstance(schema.additional_log_standard_attrs, list)
723+
assert schema.additional_log_standard_attrs == ["name"]
724+
725+
def test_additional_log_standard_attrs_type_error(self):
726+
with pytest.raises(ValidationError):
727+
LoggingConfig.parse_obj({"additional_log_standard_attrs": "name"})
728+
729+
def test_additional_log_standard_attrs_deduplicate(self):
730+
schema = LoggingConfig.parse_obj(
731+
{"additional_log_standard_attrs": ["name", "name"]}
732+
)
733+
assert schema.additional_log_standard_attrs == ["name"]
717734

718735

719736
# This function is defined globally to be accessible via import path

src/ray/protobuf/serve.proto

+1
Original file line numberDiff line numberDiff line change
@@ -86,6 +86,7 @@ message LoggingConfig {
8686
string log_level = 2;
8787
string logs_dir = 3;
8888
bool enable_access_log = 4;
89+
repeated string additional_log_standard_attrs = 5;
8990
}
9091

9192
//[End] Logging Config

0 commit comments

Comments
 (0)