Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 7857fa7

Browse files
committed
[BEAM-7109] Do not reconnect logging at termination
1 parent 9c8a8dc commit 7857fa7

1 file changed

Lines changed: 18 additions & 4 deletions

File tree

sdks/python/apache_beam/runners/worker/log_handler.py

Lines changed: 18 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import queue
2525
import sys
2626
import threading
27+
import time
2728
from builtins import range
2829

2930
import grpc
@@ -59,23 +60,25 @@ class FnApiLogRecordHandler(logging.Handler):
5960
def __init__(self, log_service_descriptor):
6061
super(FnApiLogRecordHandler, self).__init__()
6162

63+
self._alive = True
6264
self._dropped_logs = 0
6365
self._log_entry_queue = queue.Queue(maxsize=self._QUEUE_SIZE)
6466

6567
ch = GRPCChannelFactory.insecure_channel(log_service_descriptor.url)
6668
# Make sure the channel is ready to avoid [BEAM-4649]
6769
grpc.channel_ready_future(ch).result(timeout=60)
6870
self._log_channel = grpc.intercept_channel(ch, WorkerIdInterceptor())
69-
self._logging_stub = beam_fn_api_pb2_grpc.BeamFnLoggingStub(
70-
self._log_channel)
71-
7271
self._reader = threading.Thread(
7372
target=lambda: self._read_log_control_messages(),
7473
name='read_log_control_messages')
7574
self._reader.daemon = True
7675
self._reader.start()
7776

7877
def connect(self):
78+
if hasattr(self, '_logging_stub'):
79+
del self._logging_stub
80+
self._logging_stub = beam_fn_api_pb2_grpc.BeamFnLoggingStub(
81+
self._log_channel)
7982
return self._logging_stub.Logging(self._write_log_entries())
8083

8184
def emit(self, record):
@@ -96,6 +99,7 @@ def emit(self, record):
9699

97100
def close(self):
98101
"""Flush out all existing log entries and unregister this handler."""
102+
self._alive = False
99103
# Acquiring the handler lock ensures ``emit`` is not run until the lock is
100104
# released.
101105
self.acquire()
@@ -122,18 +126,28 @@ def _write_log_entries(self):
122126
yield beam_fn_api_pb2.LogEntry.List(log_entries=log_entries)
123127

124128
def _read_log_control_messages(self):
125-
while True:
129+
# Only reconnect when we are alive.
130+
# We can drop some logs in the unlikely event of logging connection
131+
# dropped(not closed) during termination when we still have logs to be sent.
132+
# This case is unlikely and the chance of reconnection and successful
133+
# transmission of logs is also very less as the process is terminating.
134+
# I choose not to handle this case to avoid un-necessary code complexity.
135+
while self._alive:
136+
# Loop for reconnection.
126137
log_control_iterator = self.connect()
127138
if self._dropped_logs > 0:
128139
logging.warn("Dropped %d logs while logging client disconnected",
129140
self._dropped_logs)
130141
self._dropped_logs = 0
131142
try:
132143
for _ in log_control_iterator:
144+
# Loop for consuming messages from server.
133145
# TODO(vikasrk): Handle control messages.
134146
pass
135147
# iterator is closed
136148
return
137149
except Exception as ex:
138150
print("Logging client failed: {}... resetting".format(ex),
139151
file=sys.stderr)
152+
# Wait a bit before trying a reconnect
153+
time.sleep(0.5) # 0.5 seconds

0 commit comments

Comments
 (0)