Skip to content

[Producer] Performance drop when 'send' is called from multiple Futures #528

@panagiks

Description

@panagiks

The following pattern (also used in benchmark/simple_produce_bench.py) is well performant:

for i in range(n):
    await producer.send(topic, message, partition=partition)

The same goes for send_and_wait, while the throughput is slightly reduced it's still highly performing.

When however the producer.send is invoked from multiple Futures there is a very steep performance drop as well as a long lasting high CPU utilization accompanied by eventual KafkaTimeoutErrors.

for i in range(n):
    asyncio.ensure_future(producer.send(topic, message, partition=partition))

Of course the above is a very over-simplified example and if you are able to call all the producer.sends centrally (as in the examples) you should always go with the first but the second is used to illustrate the case were already existing Futures eventually call producer.send.

The performance deteriorates as the number of futures trying to send at the same time increases.

I am using the following modification on benchmark/simple_produce_bench.py to test and a locally setup 1-kafka 1-zookeeper setup:

$ git diff
diff --git a/benchmark/simple_produce_bench.py b/benchmark/simple_produce_bench.py
index 2bc364f..9ff380f 100644
--- a/benchmark/simple_produce_bench.py
+++ b/benchmark/simple_produce_bench.py
@@ -9,6 +9,7 @@ import random
 class Benchmark:
 
     def __init__(self, args):
+        self._done = 0
         self._num = args.num
         self._size = args.size
         self._topic = args.topic
@@ -59,7 +60,15 @@ class Benchmark:
                 )
             )
 
+    async def send(self, producer, topic, payload, partition):
+        await producer.send(topic, payload, partition=partition)
+        self._stats[-1]['count'] += 1
+        self._done += 1
+        if self._done >= self._num:
+            self.done.set_result(None)
+
     async def bench_simple(self):
+        self.done = asyncio.Future()
         payload = bytearray(b"m" * self._size)
         topic = self._topic
         partition = self._partition
@@ -75,9 +84,9 @@ class Benchmark:
         try:
             if not self._is_transactional:
                 for i in range(self._num):
+                    asyncio.ensure_future(self.send(producer, topic, payload, partition))
                     # payload[i % self._size] = random.randint(0, 255)
-                    await producer.send(topic, payload, partition=partition)
-                    self._stats[-1]['count'] += 1
+                await self.done
             else:
                 for i in range(self._num // transaction_size):
                     # payload[i % self._size] = random.randint(0, 255)

A few test runs at low number of futures:

$ python benchmark/simple_produce_bench.py -s 200 -n 500
Total produced 500 messages in 0.05 second(s). Avg 9680.0 m/s
$ python benchmark/simple_produce_bench.py -s 200 -n 1000
Total produced 1000 messages in 0.16 second(s). Avg 6125.0 m/s
$ python benchmark/simple_produce_bench.py -s 200 -n 2000
Total produced 2000 messages in 0.70 second(s). Avg 2867.0 m/s
$ python benchmark/simple_produce_bench.py -s 200 -n 3000
Produced 1170 messages in 1 second(s).
Total produced 3000 messages in 1.68 second(s). Avg 1789.0 m/s

Already a gradual drop is visible but this becomes more clear when increasing the number of futures:

$ python benchmark/simple_produce_bench.py -s 200 -n 10000
Produced 312 messages in 1 second(s).
Produced 312 messages in 1 second(s).
[...]
Produced 624 messages in 1 second(s).
Produced 1092 messages in 1 second(s).
Total produced 10000 messages in 25.29 second(s). Avg 395.0 m/s

Switching to 20k send operations will manage to produce ~4.2k messages before the rest timeout:

Total produced 4212 messages in 54.19 second(s). Avg 77.0 m/s

On the contrary, running with the original benchmark code (not from within Futures):

$ python benchmark/simple_produce_bench.py -s 200 -n 20000
Total produced 20000 messages in 0.35 second(s). Avg 57651.0 m/s

Usecase

In my usecase I have job system where I/O blocking jobs are scheduled, each one as an own Future, and after the I/O waiting part of each job is completed it produces a Kafka message. The 'traffic' of jobs is bursty with peaks reaching 20k ~ 30k jobs scheduled concurrently and lower input periods.

First Look into the issue

I had a first swing at tackling the issue by running my modified benchmark with profiling enabled.

Comparing the profiling output of the Futures approach to the one already in the benchmark file, the first thing that pops up is that message_accumulator's add_message function overshadows all else in both cumtime and tottime. Also interesting is the ncalls that shows that with the Futures approach add_message recursed heavily compared to the first approach.

First approach:

ncalls tottime percall cumtime percall filename:lineno(function)
9230/9115 0.0104 1.141e-06 0.1546 1.696e-05 message_accumulator.py:310(add_message)

With Futures:

ncalls tottime percall cumtime percall filename:lineno(function)
20493720/523740 6.781 1.295e-05 25.45 4.86e-05 message_accumulator.py:310(add_message)

The per-call time also seems to have increased significantly (takes ~4x) but in the range of values we have this could be insignificant.

Unfortunately, that's how far I managed to get. I was not able to find (even approximately) what is that causes so heavy recursion. I mean I found that the recursion exists in the first place due to aiokafka/producer/message_accumulator.py#L341-L342 but I could not figure out why the same load of messages would trigger such a heavy recursion when coming from multiple Futures and not in the other approach.

My only guess would be that in the benchmark send (and consequently add_message) are never really called asynchronously, they are async and are awaited but they are awaited sequentially (instead of doing for example an asyncio.gather on them). This however is nothing more than a guess as I was not able to find any actual evidence pointing to this being the issue.

I used the following changes on the benchmark code to test asynchronous calls to send without wrapping each call in an additional Future and the behavior is similar to the Futures approach (i.e. there is a breakdown at ~20k messages):

$ git diff
diff --git a/benchmark/simple_produce_bench.py b/benchmark/simple_produce_bench.py
index 2bc364f..893ea9e 100644
--- a/benchmark/simple_produce_bench.py
+++ b/benchmark/simple_produce_bench.py
@@ -59,6 +59,10 @@ class Benchmark:
                 )
             )
 
+    async def send(self, producer, topic, payload, partition):
+        await producer.send(topic, payload, partition=partition)
+        self._stats[-1]['count'] += 1
+
     async def bench_simple(self):
         payload = bytearray(b"m" * self._size)
         topic = self._topic
@@ -74,10 +78,10 @@ class Benchmark:
 
         try:
             if not self._is_transactional:
-                for i in range(self._num):
-                    # payload[i % self._size] = random.randint(0, 255)
-                    await producer.send(topic, payload, partition=partition)
-                    self._stats[-1]['count'] += 1
+                await asyncio.gather(*[
+                    self.send(producer, topic, payload, partition)
+                    for i in range(self._num)
+                ])
             else:
                 for i in range(self._num // transaction_size):
                     # payload[i % self._size] = random.randint(0, 255)

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions