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

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

                Author: ASF GitHub Bot
            Created on: 19/Mar/19 18:57
            Start Date: 19/Mar/19 18:57
    Worklog Time Spent: 10m 
      Work Description: boyuanzz commented on pull request #7937: [BEAM-6778] 
Enable Bundle Finalization in Python SDK harness over FnApi
URL: https://github.com/apache/beam/pull/7937#discussion_r267049637
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -199,21 +195,26 @@ def task():
         "Currently using %s threads." % 
len(self._process_thread_pool._threads))
 
   def _request_process_bundle_split(self, request):
-    self._request_process_bundle_action(request)
+    self._request_process_bundle_action(request, self._progress_thread_pool)
+
+  def _request_finalize_bundle(self, request):
+    self._request_process_bundle_action(request, self._finalize_thread_pool)
 
   def _request_process_bundle_progress(self, request):
-    self._request_process_bundle_action(request)
+    self._request_process_bundle_action(request, self._progress_thread_pool)
 
-  def _request_process_bundle_action(self, request):
+  def _request_process_bundle_action(self, request, thread_pool):
 
     def task():
       instruction_reference = getattr(
           request, request.WhichOneof('request')).instruction_reference
-      if instruction_reference in self._instruction_id_vs_worker:
-        self._execute(
-            lambda: self._instruction_id_vs_worker[
-                instruction_reference
-            ].do_instruction(request), request)
+      if instruction_reference not in self._unscheduled_process_bundle:
+        worker = self.workers.get()
 
 Review comment:
   Based on previous discussion: 
   > The only issue is that there is a limited number of workers, and we don't 
want to use them up (or wait for them) for progress/split calls. Given that 
these worker are stateless and threadsafe, perhaps a reasonable solution would 
be to create one extra worker that we always use for progress and split (but 
not for process or bundle finalization).
   > 
   How about, creating one extra worker inside progress_pool to handle 
progress/split, creating one extra worker in process_pool to handle 
finalization. The ```self._worker_count``` will be number of workers processing 
bundle.
   
 
----------------------------------------------------------------
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: 215662)
    Time Spent: 7h 20m  (was: 7h 10m)

> Enable Bundle Finalization in Python SDK
> ----------------------------------------
>
>                 Key: BEAM-6778
>                 URL: https://issues.apache.org/jira/browse/BEAM-6778
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-py-harness
>            Reporter: Boyuan Zhang
>            Assignee: Boyuan Zhang
>            Priority: Major
>          Time Spent: 7h 20m
>  Remaining Estimate: 0h
>




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

Reply via email to