Revamp ray core design patterns doc [2/n]: too fine grained tasks (#27919)

Move the code to doc_code
Fix the code example to make batching faster than serial run.

Related issue number

#27048

Signed-off-by: Jiajun Yao <jeromeyjj@gmail.com>
This commit is contained in:
Jiajun Yao 2022-08-17 13:52:50 -07:00 committed by GitHub
parent edde905741
commit 0a3a5e68a4
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
5 changed files with 89 additions and 44 deletions

View file

@ -0,0 +1,59 @@
# __anti_pattern_start__
import ray
import time
import itertools
ray.init()
numbers = list(range(10000))
def double(number):
time.sleep(0.00001)
return number * 2
start_time = time.time()
serial_doubled_numbers = [double(number) for number in numbers]
end_time = time.time()
print(f"Ordinary funciton call takes {end_time - start_time} seconds")
# Ordinary funciton call takes 0.16506004333496094 seconds
@ray.remote
def remote_double(number):
return double(number)
start_time = time.time()
doubled_number_refs = [remote_double.remote(number) for number in numbers]
parallel_doubled_numbers = ray.get(doubled_number_refs)
end_time = time.time()
print(f"Parallelizing tasks takes {end_time - start_time} seconds")
# Parallelizing tasks takes 1.6061789989471436 seconds
# __anti_pattern_end__
assert serial_doubled_numbers == parallel_doubled_numbers
# __batching_start__
@ray.remote
def remote_double_batch(numbers):
return [double(number) for number in numbers]
BATCH_SIZE = 1000
start_time = time.time()
doubled_batch_refs = []
for i in range(0, len(numbers), BATCH_SIZE):
batch = numbers[i : i + BATCH_SIZE]
doubled_batch_refs.append(remote_double_batch.remote(batch))
parallel_doubled_numbers_with_batching = list(
itertools.chain(*ray.get(doubled_batch_refs))
)
end_time = time.time()
print(f"Parallelizing tasks with batching takes {end_time - start_time} seconds")
# Parallelizing tasks with batching takes 0.030150890350341797 seconds
# __batching_end__
assert serial_doubled_numbers == parallel_doubled_numbers_with_batching

View file

@ -9,3 +9,4 @@ This section is a collection of common design patterns and anti-patterns for wri
:maxdepth: 1
generators
too-fine-grained-tasks

View file

@ -0,0 +1,29 @@
Anti-pattern: Over-parallelizing with too fine-grained tasks harms speedup
==========================================================================
**TLDR:** Avoid over-parallelizing. Parallelizing tasks has higher overhead than using normal functions.
Parallelizing or distributing tasks usually comes with higher overhead than an ordinary function call. Therefore, if you parallelize a function that executes very quickly, the overhead could take longer than the actual function call!
To handle this problem, we should be careful about parallelizing too much. If you have a function or task thats too small, you can use a technique called **batching** to make your tasks do more meaningful work in a single call.
Code example
------------
**Anti-pattern:**
.. literalinclude:: ../doc_code/anti_pattern_too_fine_grained_tasks.py
:language: python
:start-after: __anti_pattern_start__
:end-before: __anti_pattern_end__
**Better approach:** Use batching.
.. literalinclude:: ../doc_code/anti_pattern_too_fine_grained_tasks.py
:language: python
:start-after: __batching_start__
:end-before: __batching_end__
As we can see from the example above, over-parallelizing has higher overhead and the program runs slower than the serial version.
Through batching with a proper batch size, we are able to amortize the overhead and achieve the expected speedup.

View file

@ -1,43 +0,0 @@
Antipattern: Too fine-grained tasks
===================================
**TLDR:** Avoid over-parallelizing. Parallelizing tasks has higher overhead than using normal functions.
Parallelizing or distributing tasks usually comes with higher overhead than an ordinary function call. Therefore, if you parallelize a function that executes very quickly, the overhead could take longer than the actual function call!
To handle this problem, we should be careful about parallelizing too much. If you have a function or task thats too small, you can use a technique called batching to make your tasks do more meaningful work in a single task.
Code example
------------
**Antipattern:**
.. code-block:: python
@ray.remote
def double(number):
return number * 2
numbers = list(range(10000))
doubled_numbers = []
for i in numbers:
doubled_numbers.append(ray.get(double.remote(i)))
**Better approach:** Use batching.
.. code-block:: python
@ray.remote
def double_list(list_of_numbers):
return [number * 2 for number in list_of_numbers]
numbers = list(range(10000))
doubled_list_refs = []
BATCH_SIZE = 100
for i in range(0, len(numbers), BATCH_SIZE):
batch = numbers[i : i + BATCH_SIZE]
doubled_list_refs.append(double_list.remote(batch))
doubled_numbers = []

View file

@ -17,7 +17,6 @@ You may also be interested in visiting the design patterns section for :ref:`act
map-reduce
limit-tasks
closure-capture
fine-grained-tasks
global-variables
ray-get-loop
submission-order