Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@

* Support configuring Firestore database on ReadFn transforms (Java) ([#36904](https://github.com/apache/beam/issues/36904)).
* (Python) Inference args are now allowed in most model handlers, except where they are explicitly/intentionally disallowed ([#37093](https://github.com/apache/beam/issues/37093)).
* Include logger name in FnApiLogRecordHandler log entries for filtering and debugging (Python) ([#37146](https://github.com/apache/beam/issues/37146)).

## Breaking Changes

Expand Down
4 changes: 4 additions & 0 deletions sdks/python/apache_beam/runners/worker/log_handler.py
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,10 @@ def emit(self, record: logging.LogRecord) -> None:
current_state.name_context.transform_id):
log_entry.transform_id = current_state.name_context.transform_id

# Include the logger name in custom_data for filtering and debugging.
if record.name:
log_entry.custom_data.fields['logger'].string_value = record.name

try:
self._log_entry_queue.put(log_entry, block=False)
except queue.Full:
Expand Down
9 changes: 9 additions & 0 deletions sdks/python/apache_beam/runners/worker/log_handler_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,15 @@ def test_context(self):
finally:
statesampler.set_current_tracker(None)

def test_logger_name_in_custom_data(self):
"""Tests that logger name is included in custom_data."""
_LOGGER.info('test message')
self.fn_log_handler.close()

log_entry = self.test_logging_service.log_records_received[0].log_entries[0]
self.assertEqual(
log_entry.custom_data.fields['logger'].string_value, __name__)

def test_extracts_transform_id_during_exceptions(self):
"""Tests that transform ids are captured during user code exceptions."""
descriptor = beam_fn_api_pb2.ProcessBundleDescriptor()
Expand Down
Loading