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

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

                Author: ASF GitHub Bot
            Created on: 10/Jan/19 00:28
            Start Date: 10/Jan/19 00:28
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on pull request #7433: [BEAM-6280] 
Refactors Python portability tests to be multi-threaded aware
URL: https://github.com/apache/beam/pull/7433#discussion_r246596253
 
 

 ##########
 File path: sdks/python/apache_beam/runners/portability/local_job_service.py
 ##########
 @@ -185,55 +152,89 @@ class BeamJob(threading.Thread):
     The current state of the pipeline is available as self.state.
     """
 
-  def __init__(self,
-               job_id,
-               pipeline_options,
-               pipeline_proto):
+  def __init__(self, pipeline_proto):
     super(BeamJob, self).__init__()
-    self._job_id = job_id
-    self._pipeline_options = pipeline_options
     self._pipeline_proto = pipeline_proto
     self._state = None
-    self._state_change_callbacks = []
-    self._last_log_message = None
-    self._log_callbacks = [lambda msg: setattr(self, '_last_log_message', msg)]
+    self._logs = []
+    self._final_log_count = -1
+    self._state_changes = []
     self.state = beam_job_api_pb2.JobState.STARTING
-    self.daemon = True
-
-  def add_state_change_callback(self, f):
-    self._state_change_callbacks.append(f)
-    f(self.state)
-
-  def add_log_callback(self, f):
-    self._log_callbacks.append(f)
 
   @property
   def state(self):
     return self._state
 
   @state.setter
   def state(self, new_state):
-    for state_change_callback in self._state_change_callbacks:
-      state_change_callback(new_state)
+    """Sets the job state.
+
+    This will inform GetStateStream and GetMessageStream of the new state.
+    """
+
+    self._state_changes.append(new_state)
+    self._logs.append(
+        beam_job_api_pb2.JobMessagesResponse(
+            state_response=beam_job_api_pb2.GetJobStateResponse(
+                state=new_state)))
     self._state = new_state
 
+  def _cleanup(self):
+    self._final_log_count = len(self._logs)
+
   def run(self):
-    with JobLogHandler(self._log_callbacks):
+    with JobLogHandler(self._logs):
       try:
         fn_api_runner.FnApiRunner().run_via_runner_api(self._pipeline_proto)
         logging.info('Successfully completed job.')
         self.state = beam_job_api_pb2.JobState.DONE
       except:  # pylint: disable=bare-except
         logging.exception('Error running pipeline.')
-        traceback.print_exc()
+        logging.exception(traceback.format_exc())
         self.state = beam_job_api_pb2.JobState.FAILED
         raise
+      finally:
+        # In order for consumers to read all messages, this must be the final
+        # instruction after a terminal state.
+        self._cleanup()
 
   def cancel(self):
     if self.state not in TERMINAL_STATES:
       self.state = beam_job_api_pb2.JobState.CANCELLING
       # TODO(robertwb): Actually cancel...
       self.state = beam_job_api_pb2.JobState.CANCELLED
+      self._cleanup()
+
+  def GetStateStream(self):
+    """Returns all past and future states.
+
+    This method guarentees that the consumer will see all job state 
transitions.
+    """
+    state_index = 0
+
+    # Pull all state changes until the job finishes.
+    while self.state not in TERMINAL_STATES:
+      while state_index < len(self._state_changes):
+        state = self._state_changes[state_index]
+        yield beam_job_api_pb2.GetJobStateResponse(state=state)
+        state_index += 1
+    yield beam_job_api_pb2.GetJobStateResponse(state=self.state)
+
+  def GetMessageStream(self):
+    """Returns all past and future messages.
+
+    This method guarentees that the consumer will see all messages the job
+    generates until it terminates.
+    """
+    log_index = 0
+
+    # Subscribers start with the first message and incrementally yield
+    # subsequent logs. This process repeats until the job terminates and we 
know
+    # the final amount of logs generated.
+    while log_index != self._final_log_count:
 
 Review comment:
   This will exhaust cpu (loop without any work/wait/sleep) if we have no 
pending logs (`log_index == len(self._logs)`).
 
----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 183434)
    Time Spent: 1.5h  (was: 1h 20m)

> Failure in PortableRunnerTest.test_error_traceback_includes_user_code
> ---------------------------------------------------------------------
>
>                 Key: BEAM-6280
>                 URL: https://issues.apache.org/jira/browse/BEAM-6280
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core, test-failures
>            Reporter: Kenneth Knowles
>            Assignee: Sam Rohde
>            Priority: Critical
>              Labels: flaky-test
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> [https://builds.apache.org/job/beam_PreCommit_Python_Cron/732/]
> [https://builds.apache.org/job/beam_PreCommit_Python_Cron/732/testReport/apache_beam.runners.portability.portable_runner_test/PortableRunnerTest/test_error_traceback_includes_user_code/]
> [https://scans.gradle.com/s/do3hjulee3gaa/console-log?task=:beam-sdks-python:testPython3]
> {code:java}
> 'second' not found in 'Traceback (most recent call last):\n  File 
> "/home/jenkins/jenkins-slave/workspace/beam_PreCommit_Python_Cron/src/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py",
>  line 466, in test_error_traceback_includes_user_code\n    p | 
> beam.Create([0]) | beam.Map(first)  # pylint: 
> disable=expression-not-assigned\n  File 
> "/home/jenkins/jenkins-slave/workspace/beam_PreCommit_Python_Cron/src/sdks/python/apache_beam/pipeline.py",
>  line 425, in __exit__\n    self.run().wait_until_finish()\n  File 
> "/home/jenkins/jenkins-slave/workspace/beam_PreCommit_Python_Cron/src/sdks/python/apache_beam/runners/portability/portable_runner.py",
>  line 314, in wait_until_finish\n    self._job_id, self._state, 
> self._last_error_message()))\nRuntimeError: Pipeline 
> job-cdcefe6d-1caa-4487-9e63-e971f67ec68c failed in state FAILED: start 
> <DataInputOperation receivers=[ConsumerSet[Create/Read/Impulse.out0, 
> coder=WindowedValueCoder[BytesCoder], len(consumers)=1]]>\n'{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to