forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathworker_status.py
More file actions
324 lines (280 loc) · 11.4 KB
/
worker_status.py
File metadata and controls
324 lines (280 loc) · 11.4 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
"""Worker status api handler for reporting SDK harness debug info."""
import gc
import logging
import queue
import sys
import threading
import time
import traceback
from collections import defaultdict
import grpc
from apache_beam.portability.api import beam_fn_api_pb2
from apache_beam.portability.api import beam_fn_api_pb2_grpc
from apache_beam.runners.worker.channel_factory import GRPCChannelFactory
from apache_beam.runners.worker.statecache import StateCache
from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor
from apache_beam.utils.sentinel import Sentinel
try:
from guppy import hpy
except ImportError:
hpy = None
_LOGGER = logging.getLogger(__name__)
# This SDK harness will (by default), log a "lull" in processing if it sees no
# transitions in over 5 minutes.
# 5 minutes * 60 seconds * 1000 millis * 1000 micros * 1000 nanoseconds
DEFAULT_LOG_LULL_TIMEOUT_NS = 5 * 60 * 1000 * 1000 * 1000
# Full thread dump is performed at most every 20 minutes.
LOG_LULL_FULL_THREAD_DUMP_INTERVAL_S = 20 * 60
# Full thread dump is performed if the lull is more than 20 minutes.
LOG_LULL_FULL_THREAD_DUMP_LULL_S = 20 * 60
def _current_frames():
# Work around https://github.com/python/cpython/issues/106883
if (sys.version_info.minor == 11 and sys.version_info.major == 3 and
gc.isenabled()):
gc.disable()
frames = sys._current_frames() # pylint: disable=protected-access
gc.enable()
return frames
else:
return sys._current_frames() # pylint: disable=protected-access
def thread_dump():
"""Get a thread dump for the current SDK worker harness. """
# deduplicate threads with same stack trace
stack_traces = defaultdict(list)
frames = _current_frames()
for t in threading.enumerate():
try:
stack_trace = ''.join(traceback.format_stack(frames[t.ident]))
except KeyError:
# the thread may have been destroyed already while enumerating, in such
# case, skip to next thread.
continue
thread_ident_name = (t.ident, t.name)
stack_traces[stack_trace].append(thread_ident_name)
all_traces = ['=' * 10 + ' THREAD DUMP ' + '=' * 10]
for stack, identity in stack_traces.items():
ident, name = identity[0]
trace = '--- Thread #%s name: %s %s---\n' % (
ident,
name,
'and other %d threads' %
(len(identity) - 1) if len(identity) > 1 else '')
if len(identity) > 1:
trace += 'threads: %s\n' % identity
trace += stack
all_traces.append(trace)
all_traces.append('=' * 30)
return '\n'.join(all_traces)
def heap_dump():
"""Get a heap dump for the current SDK worker harness. """
banner = '=' * 10 + ' HEAP DUMP ' + '=' * 10 + '\n'
if not hpy:
heap = 'Unable to import guppy, the heap dump will be skipped.\n'
else:
heap = '%s\n' % hpy().heap()
ending = '=' * 30
return banner + heap + ending
def _state_cache_stats(state_cache: StateCache) -> str:
"""Gather state cache statistics."""
cache_stats = ['=' * 10 + ' CACHE STATS ' + '=' * 10]
if not state_cache.is_cache_enabled():
cache_stats.append("Cache disabled")
else:
cache_stats.append(state_cache.describe_stats())
return '\n'.join(cache_stats)
def _active_processing_bundles_state(bundle_process_cache):
"""Gather information about the currently in-processing active bundles.
The result only keeps the longest lasting 10 bundles to avoid excessive
spamming.
"""
active_bundles = ['=' * 10 + ' ACTIVE PROCESSING BUNDLES ' + '=' * 10]
if not bundle_process_cache.active_bundle_processors:
active_bundles.append("No active processing bundles.")
else:
cache = []
for instruction in list(
bundle_process_cache.active_bundle_processors.keys()):
processor = bundle_process_cache.lookup(instruction)
if processor:
info = processor.state_sampler.get_info()
cache.append((
instruction,
processor.process_bundle_descriptor.id,
info.tracked_thread,
info.time_since_transition))
# reverse sort active bundle by time since last transition, keep top 10.
cache.sort(key=lambda x: x[-1], reverse=True)
for s in cache[:10]:
state = '--- instruction %s ---\n' % s[0]
state += 'ProcessBundleDescriptorId: %s\n' % s[1]
state += "tracked thread: %s\n" % s[2]
state += "time since transition: %.2f seconds\n" % (s[3] / 1e9)
active_bundles.append(state)
active_bundles.append('=' * 30)
return '\n'.join(active_bundles)
DONE = Sentinel.sentinel
class FnApiWorkerStatusHandler(object):
"""FnApiWorkerStatusHandler handles worker status request from Runner. """
def __init__(
self,
status_address,
bundle_process_cache=None,
state_cache=None,
enable_heap_dump=False,
worker_id=None,
log_lull_timeout_ns=DEFAULT_LOG_LULL_TIMEOUT_NS,
element_processing_timeout_minutes=None):
"""Initialize FnApiWorkerStatusHandler.
Args:
status_address: The URL Runner uses to host the WorkerStatus server.
bundle_process_cache: The BundleProcessor cache dict from sdk worker.
state_cache: The StateCache form sdk worker.
"""
self._alive = True
self._bundle_process_cache = bundle_process_cache
self._state_cache = state_cache
ch = GRPCChannelFactory.insecure_channel(status_address)
grpc.channel_ready_future(ch).result(timeout=60)
self._status_channel = grpc.intercept_channel(
ch, WorkerIdInterceptor(worker_id))
self._status_stub = beam_fn_api_pb2_grpc.BeamFnWorkerStatusStub(
self._status_channel)
self._responses = queue.Queue()
self.log_lull_timeout_ns = log_lull_timeout_ns
if element_processing_timeout_minutes:
self._element_processing_timeout_ns = element_processing_timeout_minutes * 60 * 1e9
else:
self._element_processing_timeout_ns = None
self._last_full_thread_dump_secs = 0.0
self._last_lull_logged_secs = 0.0
self._server = threading.Thread(
target=lambda: self._serve(), name='fn_api_status_handler')
self._server.daemon = True
self._enable_heap_dump = enable_heap_dump
self._server.start()
self._lull_logger = threading.Thread(
target=lambda: self._log_lull_in_bundle_processor(
self._bundle_process_cache),
name='lull_operation_logger')
self._lull_logger.daemon = True
self._lull_logger.start()
def _get_responses(self):
while True:
response = self._responses.get()
if response is DONE:
self._alive = False
return
yield response
def _serve(self):
while self._alive:
for request in self._status_stub.WorkerStatus(self._get_responses()):
try:
self._responses.put(
beam_fn_api_pb2.WorkerStatusResponse(
id=request.id, status_info=self.generate_status_response()))
except Exception:
traceback_string = traceback.format_exc()
self._responses.put(
beam_fn_api_pb2.WorkerStatusResponse(
id=request.id,
error="Exception encountered while generating "
"status page: %s" % traceback_string))
def generate_status_response(self):
all_status_sections = []
if self._state_cache:
all_status_sections.append(_state_cache_stats(self._state_cache))
if self._bundle_process_cache:
all_status_sections.append(
_active_processing_bundles_state(self._bundle_process_cache))
all_status_sections.append(thread_dump())
if self._enable_heap_dump:
all_status_sections.append(heap_dump())
return '\n'.join(all_status_sections)
def close(self):
self._responses.put(DONE, timeout=5)
def _log_lull_in_bundle_processor(self, bundle_process_cache):
while True:
time.sleep(2 * 60)
if bundle_process_cache and bundle_process_cache.active_bundle_processors:
for instruction in list(
bundle_process_cache.active_bundle_processors.keys()):
processor = bundle_process_cache.lookup(instruction)
if processor:
info = processor.state_sampler.get_info()
self._log_lull_sampler_info(info, instruction)
if self._element_processing_timeout_ns:
self._terminate_sdk_worker_lull(info, instruction)
def _terminate_sdk_worker_lull(self, sampler_info, instruction):
if (sampler_info and sampler_info.time_since_transition and
sampler_info.time_since_transition
> self._element_processing_timeout_ns):
lull_seconds = sampler_info.time_since_transition / 1e9
step_name = sampler_info.state_name.step_name
state_name = sampler_info.state_name.name
if step_name and state_name:
step_name_log = (
' for PTransform{name=%s, state=%s}' % (step_name, state_name))
else:
step_name_log = ''
stack_trace = self._get_stack_trace(sampler_info)
log_lull_msg = (
'Operation ongoing in bundle %s%s for at least %.2f seconds'
' without outputting or completing.\n'
'Current Traceback:\n%s.' %
(instruction, step_name_log, lull_seconds, stack_trace))
raise TimeoutError(log_lull_msg + 'The SDK harness will be terminated.')
def _log_lull_sampler_info(self, sampler_info, instruction):
if not self._passed_lull_timeout_since_last_log():
return
if (sampler_info and sampler_info.time_since_transition and
sampler_info.time_since_transition > self.log_lull_timeout_ns):
lull_seconds = sampler_info.time_since_transition / 1e9
step_name = sampler_info.state_name.step_name
state_name = sampler_info.state_name.name
if step_name and state_name:
step_name_log = (
' for PTransform{name=%s, state=%s}' % (step_name, state_name))
else:
step_name_log = ''
stack_trace = self._get_stack_trace(sampler_info)
_LOGGER.warning(
(
'Operation ongoing in bundle %s%s for at least %.2f seconds'
' without outputting or completing.\n'
'Current Traceback:\n%s'),
instruction,
step_name_log,
lull_seconds,
stack_trace,
)
def _get_stack_trace(self, sampler_info):
exec_thread = getattr(sampler_info, 'tracked_thread', None)
if exec_thread is not None:
thread_frame = _current_frames().get(exec_thread.ident)
return '\n'.join(
traceback.format_stack(thread_frame)) if thread_frame else ''
else:
return '-NOT AVAILABLE-'
def _passed_lull_timeout_since_last_log(self) -> bool:
if (time.time() - self._last_lull_logged_secs
> self.log_lull_timeout_ns / 1e9):
self._last_lull_logged_secs = time.time()
return True
else:
return False