Yichi Zhang created BEAM-8944:
---------------------------------

             Summary: 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.17.0, 2.18.0
            Reporter: Yichi Zhang


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:

 

```

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)


``` 

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
```



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

Reply via email to