confluentinc / librdkafka

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

Data race in `rd_kafka_stats_emit_all` #4522

Open alexey-milovidov opened 10 months ago

alexey-milovidov commented 10 months ago

Description

WARNING: ThreadSanitizer: data race (pid=1)
  Atomic write of size 4 at 0x7b78000f4bb0 by thread T377 (mutexes: read M0, write M1):
    #0 rd_atomic32_get build_docker/./contrib/librdkafka/src/rdatomic.h:108:9 (clickhouse+0x1f8b1a93) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #1 rd_kafka_stats_emit_all build_docker/./contrib/librdkafka/src/rdkafka.c:1654:3 (clickhouse+0x1f8b1a93)
    #2 rd_kafka_stats_emit_tmr_cb build_docker/./contrib/librdkafka/src/rdkafka.c:1898:2 (clickhouse+0x1f8b1a93)
    #3 rd_kafka_timers_run build_docker/./contrib/librdkafka/src/rdkafka_timer.c:288:4 (clickhouse+0x1f9a29aa) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #4 rd_kafka_thread_main build_docker/./contrib/librdkafka/src/rdkafka.c:2021:3 (clickhouse+0x1f8a8b29) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #5 _thrd_wrapper_function build_docker/./contrib/librdkafka/src/tinycthread.c:576:9 (clickhouse+0x1f9c611b) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)

  Previous write of size 4 at 0x7b78000f4bb0 by thread T376:
    #0 rd_atomic32_init build_docker/./contrib/librdkafka/src/rdatomic.h:49:10 (clickhouse+0x1f8a3173) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #1 rd_kafka_bufq_init build_docker/./contrib/librdkafka/src/rdkafka_buf.c:225:2 (clickhouse+0x1f8a3173)
    #2 rd_kafka_bufq_concat build_docker/./contrib/librdkafka/src/rdkafka_buf.c:236:2 (clickhouse+0x1f8a3173)
    #3 rd_kafka_broker_fail build_docker/./contrib/librdkafka/src/rdkafka_broker.c:572:2 (clickhouse+0x1f87a82f) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #4 rd_kafka_broker_op_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:3317:33 (clickhouse+0x1f88bc18) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #5 rd_kafka_broker_ops_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:3351:24 (clickhouse+0x1f88bc18)
    #6 rd_kafka_broker_ops_io_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:3401:9 (clickhouse+0x1f88e323) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #7 rd_kafka_broker_consumer_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:4975:17 (clickhouse+0x1f88a0be) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #8 rd_kafka_broker_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:5080:17 (clickhouse+0x1f88a0be)
    #9 rd_kafka_broker_thread_main build_docker/./contrib/librdkafka/src/rdkafka_broker.c:5237:25 (clickhouse+0x1f8834b9) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)
    #10 _thrd_wrapper_function build_docker/./contrib/librdkafka/src/tinycthread.c:576:9 (clickhouse+0x1f9c611b) (BuildId: 662ff0bfd164437162d27a08d006d6ba54fefcf7)

SUMMARY: ThreadSanitizer: data race build_docker/./contrib/librdkafka/src/rdatomic.h:108:9 in rd_atomic32_get

https://github.com/ClickHouse/ClickHouse/issues/56043

How to reproduce

Run ClickHouse integration tests with TSan until it reproduces.

Checklist

IMPORTANT: We will close issues where the checklist has not been completed.

Please provide the following information:

antaljanosbenjamin commented 10 months ago

@ilejn found the probably cause for this datarace:

One thread gets counter in statistic callback, another thread sets this counter to zero because of rebalance.

emasab commented 10 months ago

Yeah, it should be solved by using the atomic rd_atomic32_set instead of rd_atomic32_init, as the source buffer isn't being used for the first time, it's either the rkb->rkb_waitresps or rkb->rkb_outbufs and those buffers are being concatenated to a temp buffer queue.

ilejn commented 9 months ago

Yeah, it should be solved by using the atomic rd_atomic32_set instead of rd_atomic32_init, as the source buffer isn't being used for the first time, it's either the rkb->rkb_waitresps or rkb->rkb_outbufs and those buffers are being concatenated to a temp buffer queue.

Yes, thanks @emasab , this is correct until the branch in rd_atomic32_set is

    return ra->val = v; // FIXME
#endif

Fortunately we do have HAVE_ATOMICS_32_ATOMIC defined in ClickHouse :)

nhaq-confluent commented 8 months ago

@ilejn Can this issue be closed?

ilejn commented 8 months ago

@nhaq-confluent Can this issue be closed?

Hello @nhaq-confluent , the problem is fixed in ClickHouse librdkafka fork https://github.com/ClickHouse/librdkafka/pull/10 but as far as I can see it is not fixed in confluentinc/librdkafka master. Honestly I was sure that I had created a PR in confluentinc/librdkafka, but it seems that it is not true. Should I?

nhaq-confluent commented 8 months ago

@ilejn if you can, that would be useful. We are in the process of closing out some of our issues and PR's. The PR would be easier for us to resolve as we go through everything.