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_broker_timeout_scan #4625

Open
2 of 7 tasks
antaljanosbenjamin opened this issue Feb 27, 2024 · 0 comments · May be fixed by #4630
Open
2 of 7 tasks

Data race in rd_kafka_broker_timeout_scan #4625

antaljanosbenjamin opened this issue Feb 27, 2024 · 0 comments · May be fixed by #4630
Labels

Comments

@antaljanosbenjamin
Copy link

Read the FAQ first: https://github.com/confluentinc/librdkafka/wiki/FAQ

Do NOT create issues for questions, use the discussion forum: https://github.com/confluentinc/librdkafka/discussions

Description

Exception: Sanitizer assert found for instance �==================
WARNING: ThreadSanitizer: data race (pid=1)
  Read of size 8 at 0x7b7800127158 by thread T987 (mutexes: read M0, write M1, write M2):
    #0 __tsan_memcpy <null> (clickhouse+0x74eebb7) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #1 rd_avg_rollover build_docker/./contrib/librdkafka/src/rdavg.h:153:22 (clickhouse+0x1e39753b) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #2 rd_kafka_stats_emit_avg build_docker/./contrib/librdkafka/src/rdkafka.c:1354:9 (clickhouse+0x1e39753b)
    #3 rd_kafka_stats_emit_all build_docker/./contrib/librdkafka/src/rdkafka.c:1717:17 (clickhouse+0x1e395c8b) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #4 rd_kafka_stats_emit_tmr_cb build_docker/./contrib/librdkafka/src/rdkafka.c:1898:2 (clickhouse+0x1e395c8b)
    #5 rd_kafka_timers_run build_docker/./contrib/librdkafka/src/rdkafka_timer.c:288:4 (clickhouse+0x1e46498a) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #6 rd_kafka_thread_main build_docker/./contrib/librdkafka/src/rdkafka.c:2021:3 (clickhouse+0x1e3919e9) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #7 _thrd_wrapper_function build_docker/./contrib/librdkafka/src/tinycthread.c:576:9 (clickhouse+0x1e47a57b) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)

  Previous write of size 8 at 0x7b7800127158 by thread T986:
    #0 rd_avg_calc build_docker/./contrib/librdkafka/src/rdavg.h:104:38 (clickhouse+0x1e37d71d) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #1 rd_kafka_broker_timeout_scan build_docker/./contrib/librdkafka/src/rdkafka_broker.c:880:25 (clickhouse+0x1e37d71d)
    #2 rd_kafka_broker_ops_io_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:3416:17 (clickhouse+0x1e37d71d)
    #3 rd_kafka_broker_consumer_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:4975:17 (clickhouse+0x1e378e5e) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #4 rd_kafka_broker_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:5080:17 (clickhouse+0x1e378e5e)
    #5 rd_kafka_broker_thread_main build_docker/./contrib/librdkafka/src/rdkafka_broker.c:5237:25 (clickhouse+0x1e372619) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
    #6 _thrd_wrapper_function build_docker/./contrib/librdkafka/src/tinycthread.c:576:9 (clickhouse+0x1e47a57b) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)

ClickHouse/ClickHouse#60443

How to reproduce

Run ClickHouse integration tests with TSan until it reproduces.

Checklist

Please provide the following information:

  • librdkafka version: https://github.com/ClickHouse/librdkafka/commits/6f3b483426a8c8ec950e27e446bec175cf8b553f/, but I think the issue is also present in recent master.
  • 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: <REPLACE with e.g., Centos 5 (x64)>
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue

I think the issue is in rd_kafka_broker_timeout_scan accesses rkb->rkb_avg_rtt without holding a lock for rkb. In rd_kafka_stats_emit_all a read lock is taken when the stats are modified. However I am not sure how to properly fix it.

@antaljanosbenjamin antaljanosbenjamin changed the title Data race in rd_kafka_broker_timeout_scan/rd_avg_calc Data race in rd_kafka_broker_timeout_scan Feb 27, 2024
@azat azat linked a pull request Feb 29, 2024 that will close this issue
@emasab emasab added the bug label Mar 20, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants