[ https://issues.apache.org/jira/browse/BEAM-5633?focusedWorklogId=152906&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-152906 ]
ASF GitHub Bot logged work on BEAM-5633: ---------------------------------------- Author: ASF GitHub Bot Created on: 09/Oct/18 20:04 Start Date: 09/Oct/18 20:04 Worklog Time Spent: 10m Work Description: aaltay closed pull request #6593: [BEAM-5633] Adds reconnect logic to python logging client URL: https://github.com/apache/beam/pull/6593 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/sdks/python/apache_beam/runners/worker/log_handler.py b/sdks/python/apache_beam/runners/worker/log_handler.py index ad6e20b17d2..f72c7b00779 100644 --- a/sdks/python/apache_beam/runners/worker/log_handler.py +++ b/sdks/python/apache_beam/runners/worker/log_handler.py @@ -17,10 +17,12 @@ """Beam fn API log handler.""" from __future__ import absolute_import +from __future__ import print_function import logging import math import queue +import sys import threading from builtins import range @@ -40,6 +42,9 @@ class FnApiLogRecordHandler(logging.Handler): _MAX_BATCH_SIZE = 1000 # Used to indicate the end of stream. _FINISHED = object() + # Size of the queue used to buffer messages. Once full, messages will be + # dropped. If the average log size is 1KB this may use up to 10MB of memory. + _QUEUE_SIZE = 10000 # Mapping from logging levels to LogEntry levels. LOG_LEVEL_MAP = { @@ -52,21 +57,26 @@ class FnApiLogRecordHandler(logging.Handler): def __init__(self, log_service_descriptor): super(FnApiLogRecordHandler, self).__init__() - # Make sure the channel is ready to avoid [BEAM-4649] + + self._dropped_logs = 0 + self._log_entry_queue = queue.Queue(maxsize=self._QUEUE_SIZE) + ch = grpc.insecure_channel(log_service_descriptor.url) + # Make sure the channel is ready to avoid [BEAM-4649] grpc.channel_ready_future(ch).result(timeout=60) self._log_channel = grpc.intercept_channel(ch, WorkerIdInterceptor()) self._logging_stub = beam_fn_api_pb2_grpc.BeamFnLoggingStub( self._log_channel) - self._log_entry_queue = queue.Queue() - log_control_messages = self._logging_stub.Logging(self._write_log_entries()) self._reader = threading.Thread( - target=lambda: self._read_log_control_messages(log_control_messages), + target=lambda: self._read_log_control_messages(), name='read_log_control_messages') self._reader.daemon = True self._reader.start() + def connect(self): + return self._logging_stub.Logging(self._write_log_entries()) + def emit(self, record): log_entry = beam_fn_api_pb2.LogEntry() log_entry.severity = self.LOG_LEVEL_MAP[record.levelno] @@ -77,14 +87,18 @@ def emit(self, record): nanoseconds = 1e9 * fraction log_entry.timestamp.seconds = int(seconds) log_entry.timestamp.nanos = int(nanoseconds) - self._log_entry_queue.put(log_entry) + + try: + self._log_entry_queue.put(log_entry, block=False) + except queue.Full: + self._dropped_logs += 1 def close(self): """Flush out all existing log entries and unregister this handler.""" # Acquiring the handler lock ensures ``emit`` is not run until the lock is # released. self.acquire() - self._log_entry_queue.put(self._FINISHED) + self._log_entry_queue.put(self._FINISHED, timeout=5) # wait on server to close. self._reader.join() self.release() @@ -106,7 +120,19 @@ def _write_log_entries(self): if log_entries: yield beam_fn_api_pb2.LogEntry.List(log_entries=log_entries) - def _read_log_control_messages(self, log_control_iterator): - # TODO(vikasrk): Handle control messages. - for _ in log_control_iterator: - pass + def _read_log_control_messages(self): + while True: + log_control_iterator = self.connect() + if self._dropped_logs > 0: + logging.warn("Dropped %d logs while logging client disconnected", + self._dropped_logs) + self._dropped_logs = 0 + try: + for _ in log_control_iterator: + # TODO(vikasrk): Handle control messages. + pass + # iterator is closed + return + except Exception as ex: + print("Logging client failed: {}... resetting".format(ex), + file=sys.stderr) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 152906) Time Spent: 3h (was: 2h 50m) > Python SDK harness logging client failure > ----------------------------------------- > > Key: BEAM-5633 > URL: https://issues.apache.org/jira/browse/BEAM-5633 > Project: Beam > Issue Type: Bug > Components: sdk-py-harness > Reporter: Thomas Weise > Assignee: Micah Wylde > Priority: Major > Labels: portability-flink > Time Spent: 3h > Remaining Estimate: 0h > > After running a test with synthetic source for a few minutes, the logging > client fails and all subsequent log output is not forwarded to the runner. > -- This message was sent by Atlassian JIRA (v7.6.3#76005)