[ 
https://issues.apache.org/jira/browse/BEAM-8333?focusedWorklogId=325408&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-325408
 ]

ASF GitHub Bot logged work on BEAM-8333:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 09/Oct/19 00:49
            Start Date: 09/Oct/19 00:49
    Worklog Time Spent: 10m 
      Work Description: aaltay commented on pull request #9746: [BEAM-8333] 
Adding lull logging for SDK harness
URL: https://github.com/apache/beam/pull/9746#discussion_r332790861
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -403,10 +412,35 @@ def process_bundle_split(self, request, instruction_id):
           instruction_id=instruction_id,
           error='Instruction not running: %s' % instruction_id)
 
+  def _log_lull_in_bundle_processor(self, processor):
+    state_sampler = processor.state_sampler
+    sampler_info = state_sampler.get_info()
+    if (sampler_info
+        and sampler_info.time_since_transition
+        and sampler_info.time_since_transition > self.log_lull_timeout_ns):
+      step_name = sampler_info.state_name.step_name
+      state_name = sampler_info.state_name.name
+      state_lull_log = (
+          'There has been a processing lull of over %.2f seconds in state %s'
+          % (sampler_info.time_since_transition / 1e9, state_name))
+      step_name_log = (' in step %s ' % step_name) if step_name else ''
+
+      exec_thread = getattr(sampler_info, 'tracked_thread', None)
+      if exec_thread is not None:
+        thread_frame = sys._current_frames().get(exec_thread.ident)  # pylint: 
disable=protected-access
+        stack_trace = '\n'.join(
+            traceback.format_stack(thread_frame)) if thread_frame else ''
+      else:
+        stack_trace = '-NOT AVAILABLE-'
+
+      logging.warning(
+          '%s%s. Traceback:\n%s', state_lull_log, step_name_log, stack_trace)
+
   def process_bundle_progress(self, request, instruction_id):
     # It is an error to get progress for a not-in-flight bundle.
-    processor = self.bundle_processor_cache.lookup(
-        request.instruction_id)
+    processor = self.bundle_processor_cache.lookup(request.instruction_id)
+    if processor:
+      self._log_lull_in_bundle_processor(processor)
 
 Review comment:
   Do we know the performance impact? Especially since this will happen many 
worker threads.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 325408)
    Time Spent: 1h  (was: 50m)

> Python SDK Worker should log lulls with progress-reporting thread
> -----------------------------------------------------------------
>
>                 Key: BEAM-8333
>                 URL: https://issues.apache.org/jira/browse/BEAM-8333
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-py-harness
>            Reporter: Pablo Estrada
>            Assignee: Pablo Estrada
>            Priority: Major
>          Time Spent: 1h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to