Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Data race in rd_kafka_stats_emit_all #4522

Open
3 of 7 tasks
alexey-milovidov opened this issue Nov 23, 2023 · 6 comments · May be fixed by #4718
Open
3 of 7 tasks

Data race in rd_kafka_stats_emit_all #4522

alexey-milovidov opened this issue Nov 23, 2023 · 6 comments · May be fixed by #4718
Labels

Comments

@alexey-milovidov
Copy link

alexey-milovidov commented Nov 23, 2023

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

ClickHouse/ClickHouse#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:

  • librdkafka version: https://github.com/ClickHouse/librdkafka/tree/6f3b483426a8c8ec950e27e446bec175cf8b553f (sorry, it is patched, but feel free to close this issue if you don't like it)
  • Apache Kafka version: confluentinc/cp-kafka:5.2.0
  • librdkafka client configuration: <REPLACE with e.g., message.timeout.ms=123, auto.reset.offset=earliest, ..>
  • Operating system: Linux Ubuntu 22.04
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
@antaljanosbenjamin
Copy link

@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
Copy link
Collaborator

emasab commented Dec 4, 2023

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.

@emasab emasab added the bug label Dec 4, 2023
@ilejn
Copy link

ilejn commented Dec 7, 2023

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
Copy link

@ilejn Can this issue be closed?

@ilejn
Copy link

ilejn commented Feb 1, 2024

@nhaq-confluent Can this issue be closed?

Hello @nhaq-confluent ,
the problem is fixed in ClickHouse librdkafka fork ClickHouse#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
Copy link

@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.

emasab added a commit that referenced this issue May 15, 2024
instead of being initialized

A data race happened when emptying buffers of a failing broker, in its thread,
with the statistics callback in main thread gathering the buffer counts.
Solved by resetting the atomic counters instead of initializing them.
Happening since 1.x

Closes #4522
emasab added a commit that referenced this issue May 15, 2024
instead of being initialized

A data race happened when emptying buffers of a failing broker, in its thread,
with the statistics callback in main thread gathering the buffer counts.
Solved by resetting the atomic counters instead of initializing them.
Happening since 1.x

Closes #4522
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
5 participants