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

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

                Author: ASF GitHub Bot
            Created on: 18/Dec/19 01:00
            Start Date: 18/Dec/19 01:00
    Worklog Time Spent: 10m 
      Work Description: y1chi commented on issue #10387: [BEAM-8944] Change to 
use single thread in py sdk bundle progress report
URL: https://github.com/apache/beam/pull/10387#issuecomment-566819037
 
 
   > > > Would it be better to have the runner request progress less frequently?
   > > 
   > > 
   > > I think that helps too. I believe right now JRH requests every 0.1 sec. 
Not exactly sure how the frequency is picked.
   > 
   > 0.1 secs is a lot and doesn't seem right.
   
   I think this is where it is set, we can try to tune that down.
   
https://github.com/apache/beam/blob/master/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutor.java#L296
   
   Single thread should be good enough for progress report in python sdk 
harness and shouldn't incur stuckness issues, and it could also help limiting 
its impact on the bundle processing critical path.
 
----------------------------------------------------------------
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: 361236)
    Time Spent: 2h 40m  (was: 2.5h)

> Python SDK harness performance degradation with UnboundedThreadPoolExecutor
> ---------------------------------------------------------------------------
>
>                 Key: BEAM-8944
>                 URL: https://issues.apache.org/jira/browse/BEAM-8944
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-harness
>    Affects Versions: 2.18.0
>            Reporter: Yichi Zhang
>            Assignee: Yichi Zhang
>            Priority: Blocker
>         Attachments: profiling.png, profiling_one_thread.png, 
> profiling_twelve_threads.png
>
>          Time Spent: 2h 40m
>  Remaining Estimate: 0h
>
> We are seeing a performance degradation for python streaming word count load 
> tests.
>  
> After some investigation, it appears to be caused by swapping the original 
> ThreadPoolExecutor to UnboundedThreadPoolExecutor in sdk worker. Suspicion is 
> that python performance is worse with more threads on cpu-bounded tasks.
>  
> A simple test for comparing the multiple thread pool executor performance:
>  
> {code:python}
> def test_performance(self):
>    def run_perf(executor):
>      total_number = 1000000
>      q = queue.Queue()
>     def task(number):
>        hash(number)
>        q.put(number + 200)
>        return number
>     t = time.time()
>      count = 0
>      for i in range(200):
>        q.put(i)
>     while count < total_number:
>        executor.submit(task, q.get(block=True))
>        count += 1
>      print('%s uses %s' % (executor, time.time() - t))
>    with UnboundedThreadPoolExecutor() as executor:
>      run_perf(executor)
>    with futures.ThreadPoolExecutor(max_workers=1) as executor:
>      run_perf(executor)
>    with futures.ThreadPoolExecutor(max_workers=12) as executor:
>      run_perf(executor)
> {code}
> Results:
> <apache_beam.utils.thread_pool_executor.UnboundedThreadPoolExecutor object at 
> 0x7fab400dbe50> uses 268.160675049
>  <concurrent.futures.thread.ThreadPoolExecutor object at 0x7fab40096290> uses 
> 79.904583931
>  <concurrent.futures.thread.ThreadPoolExecutor object at 0x7fab400dbe50> uses 
> 191.179054976
>  ```
> Profiling:
> UnboundedThreadPoolExecutor:
>  !profiling.png! 
> 1 Thread ThreadPoolExecutor:
>  !profiling_one_thread.png! 
> 12 Threads ThreadPoolExecutor:
>  !profiling_twelve_threads.png! 



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

Reply via email to