confluentinc / librdkafka

The Apache Kafka C/C++ library
Other
204 stars 3.14k forks source link

Producer performance regression (1.8.2 from 1.7.0) #3673

Closed Traktormaster closed 2 years ago

Traktormaster commented 2 years ago

Description

The producer performance is very low when producing simple messages quickly in a loop, each in a separate idempotent transaction.

There are seemingly no directly related errors in the log. However, there are strange silent points at varying times where it seems that the library just stops everything for about one second. Here is an excerpt from the attached producer log, where almost a second goes by without anything being done:

...
%7|1642019044.085|ADDPARTS|rdkafka#producer-2| [thrd:main]: test0topic0 [0] registered with transaction
%7|1642019044.085|WAKEUP|rdkafka#producer-2| [thrd:main]: 172.31.31.3:9092/1001: Wake-up
%7|1642019044.085|WAKEUP|rdkafka#producer-2| [thrd:main]: TxnCoordinator/1001: Wake-up
%7|1642019045.063|TOPPAR|rdkafka#producer-2| [thrd:172.31.31.3:9092/bootstrap]: 172.31.31.3:9092/1001: test0topic0 [0] 1 message(s) in xmit queue (1 added from partition queue)
%7|1642019045.063|PRODUCE|rdkafka#producer-2| [thrd:172.31.31.3:9092/bootstrap]: 172.31.31.3:9092/1001: test0topic0 [0]: Produce MessageSet with 1 message(s) (75 bytes, ApiVersion 7, MsgVersion 2, MsgId 21, BaseSeq 20, PID{Id:0,Epoch:0}, uncompressed)
%7|1642019045.063|SEND|rdkafka#producer-2| [thrd:172.31.31.3:9092/bootstrap]: 172.31.31.3:9092/1001: Sent ProduceRequest (v7, 160 bytes @ 0, CorrId 27)
...

Here is a different example of the pause:

...
%7|1642019049.086|WAKEUP|rdkafka#producer-2| [thrd:app]: 172.31.31.3:9092/1001: Wake-up
%7|1642019049.086|WAKEUP|rdkafka#producer-2| [thrd:app]: TxnCoordinator/1001: Wake-up
%7|1642019049.086|ADDPARTS|rdkafka#producer-2| [thrd:main]: TxnCoordinator/1001: Registering partitions with transaction
%7|1642019050.086|SEND|rdkafka#producer-2| [thrd:TxnCoordinator]: TxnCoordinator/1001: Sent AddPartitionsToTxnRequest (v0, 85 bytes @ 0, CorrId 245)
%7|1642019050.087|RECV|rdkafka#producer-2| [thrd:TxnCoordinator]: TxnCoordinator/1001: Received AddPartitionsToTxnResponse (v0, 31 bytes, CorrId 245, rtt 1.20ms)
%7|1642019050.087|ADDPARTS|rdkafka#producer-2| [thrd:main]: test0topic0 [0] registered with transaction
...

Using the 1.7.0 library version, this does not happen. The producing loop is executed in 2 to 3 seconds. The reproduction snippet finishes between 3 and 10 seconds using 1.8.2.

I have looked all over, but I can't find a reason for this.

How to reproduce

NOTE: I'm using the python binding for convenience.

from confluent_kafka import Producer

def main():
    topic = "test0topic0"
    servers = "172.31.31.3:9092"

    prod_config = {
        "bootstrap.servers": servers,
        "transactional.id": "idem__prod__tx__perf__test0",
        "enable.idempotence": True,
        "debug": "all",
    }

    prod = Producer(**prod_config)
    prod.init_transactions()

    for i in range(128):
        prod.begin_transaction()
        prod.produce(topic, value=f"asd0_{i}".encode("utf-8"))
        prod.commit_transaction()
        prod.poll(0)
    assert prod.flush(1) == 0
    prod.poll(0)

if __name__ == '__main__':
    main()

I'm guessing that this is some threading/scheduling issue, because it varies how many times it happens during a run. Rarely (about 1 out of 10 runs) it does not happen at all.

Checklist

Please provide the following information:

Logs

broker_excerpt.log producer.log

edenhill commented 2 years ago

I reproduced this on v1.8.2, every 5-10 transactions there's a ~1s delay. On master this delay is gone, and I believe it was this fix: https://github.com/edenhill/librdkafka/pull/3798/commits/f46a2f6996e72c0ad8faa4de19d740a3f0228fd8

Thanks for a great report!