Skip to content

Possible datarace around kafka timers callback #5082

@antaljanosbenjamin

Description

@antaljanosbenjamin

Here is the finding from TSAN:

E           Exception: Sanitizer assert found for instance ==================
E           WARNING: ThreadSanitizer: data race (pid=1)
E             Write of size 8 at 0x72880006e3c0 by thread T798 (mutexes: write M0):
E               #0 rd_kafka_timer_start0 ci/tmp/build/./contrib/librdkafka/src/rdkafka_timer.c:191:29 (clickhouse+0x219e4f96) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #1 rd_kafka_metadata_fast_leader_query ci/tmp/build/./contrib/librdkafka/src/rdkafka_metadata.c:1733:17 (clickhouse+0x218fd57e) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #2 rd_kafka_toppar_leader_unavailable ci/tmp/build/./contrib/librdkafka/src/rdkafka_partition.c:2585:9 (clickhouse+0x2197c5e4) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #3 rd_kafka_fetch_reply_handle_partition_error ci/tmp/build/./contrib/librdkafka/src/rdkafka_fetcher.c:255:17 (clickhouse+0x218e70b8) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #4 rd_kafka_fetch_reply_handle_partition ci/tmp/build/./contrib/librdkafka/src/rdkafka_fetcher.c:712:17 (clickhouse+0x218e2225) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #5 rd_kafka_fetch_reply_handle ci/tmp/build/./contrib/librdkafka/src/rdkafka_fetcher.c:839:29 (clickhouse+0x218dcb37) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #6 rd_kafka_broker_fetch_reply ci/tmp/build/./contrib/librdkafka/src/rdkafka_fetcher.c:909:23 (clickhouse+0x218daaf2) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #7 rd_kafka_buf_callback ci/tmp/build/./contrib/librdkafka/src/rdkafka_buf.c:515:17 (clickhouse+0x21895ab4) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #8 rd_kafka_req_response ci/tmp/build/./contrib/librdkafka/src/rdkafka_broker.c:1974:9 (clickhouse+0x2187ea0e) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #9 rd_kafka_recv ci/tmp/build/./contrib/librdkafka/src/rdkafka_broker.c:2093:17 (clickhouse+0x2187ea0e)
E               #10 rd_kafka_transport_io_event ci/tmp/build/./contrib/librdkafka/src/rdkafka_transport.c:748:32 (clickhouse+0x219ed6f7) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #11 rd_kafka_transport_io_serve ci/tmp/build/./contrib/librdkafka/src/rdkafka_transport.c:1007:17 (clickhouse+0x219ed6f7)
E               #12 rd_kafka_broker_ops_io_serve ci/tmp/build/./contrib/librdkafka/src/rdkafka_broker.c:3670:21 (clickhouse+0x2188e0d2) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #13 rd_kafka_broker_consumer_serve ci/tmp/build/./contrib/librdkafka/src/rdkafka_broker.c:4373:21 (clickhouse+0x2188aee7) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #14 rd_kafka_broker_serve ci/tmp/build/./contrib/librdkafka/src/rdkafka_broker.c:4515:17 (clickhouse+0x2188aee7)
E               #15 rd_kafka_broker_thread_main ci/tmp/build/./contrib/librdkafka/src/rdkafka_broker.c:4715:25 (clickhouse+0x2188420b) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #16 _thrd_wrapper_function ci/tmp/build/./contrib/librdkafka/src/tinycthread.c:576:9 (clickhouse+0x219fe2db) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E           
E             Previous read of size 8 at 0x72880006e3c0 by thread T796:
E               #0 rd_kafka_timers_run ci/tmp/build/./contrib/librdkafka/src/rdkafka_timer.c:357:31 (clickhouse+0x219e5a8d) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #1 rd_kafka_thread_main ci/tmp/build/./contrib/librdkafka/src/rdkafka.c:2159:17 (clickhouse+0x2189b629) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #2 _thrd_wrapper_function ci/tmp/build/./contrib/librdkafka/src/tinycthread.c:576:9 (clickhouse+0x219fe2db) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E           
E             Location is heap block of size 5768 at 0x72880006d800 allocated by thread T4:
E               #0 calloc <null> (clickhouse+0x88a6c2b) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #1 rd_calloc ci/tmp/build/./contrib/librdkafka/src/rd.h:136:19 (clickhouse+0x21898cfc) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #2 rd_kafka_new ci/tmp/build/./contrib/librdkafka/src/rdkafka.c:2249:14 (clickhouse+0x21898cfc)
E               #3 cppkafka::Consumer::Consumer(cppkafka::Configuration) ci/tmp/build/./contrib/cppkafka/src/consumer.cpp:65:23 (clickhouse+0x21870b81) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #4 cppkafka::Consumer* std::__1::construct_at[abi:ne190107]<cppkafka::Consumer, cppkafka::Configuration, cppkafka::Consumer*>(cppkafka::Consumer*, cppkafka::Configuration&&) ci/tmp/build/./contrib/llvm-project/libcxx/include/__memory/construct_at.h:41:46 (clickhouse+0x174b4417) (BuildId: 41b710068cd744a52f4301a5bc4bb7b27d807aef)
E               #5 cppkafka::Consumer* std::__1::__construct_at[abi:ne190107]<cppkafka::Consumer, cppkafka::Configuration, cppkafk

The stack is from our fork https://github.com/ClickHouse/librdkafka but as far as I know we didn't changed the code around timers.

Futhermore, I think it is a completely valid issue, because rd_kafka_timer_start0 modifies rd_kafka_timer_t::rtmr_callback while rd_kafka_timers_run reads the callback without holding the lock.

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions