Skip to content

Fix possible data-race for statistics#11

Merged
antaljanosbenjamin merged 1 commit intoClickHouse:masterfrom
azat-archive:ch/fix-data-race-in-stat
Jul 10, 2024
Merged

Fix possible data-race for statistics#11
antaljanosbenjamin merged 1 commit intoClickHouse:masterfrom
azat-archive:ch/fix-data-race-in-stat

Conversation

@azat
Copy link
Copy Markdown
Member

@azat azat commented Jul 9, 2024

TSan report (founded by ClickHouse CI):

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)

Refs: ClickHouse/ClickHouse#60443
Refs: ClickHouse/ClickHouse#60939
Refs: confluentinc#4625
Upstream PR: confluentinc#4630

TSan report (founded by ClickHouse CI):

    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)
        confluentinc#1 rd_avg_rollover build_docker/./contrib/librdkafka/src/rdavg.h:153:22 (clickhouse+0x1e39753b) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
        confluentinc#2 rd_kafka_stats_emit_avg build_docker/./contrib/librdkafka/src/rdkafka.c:1354:9 (clickhouse+0x1e39753b)
        confluentinc#3 rd_kafka_stats_emit_all build_docker/./contrib/librdkafka/src/rdkafka.c:1717:17 (clickhouse+0x1e395c8b) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
        confluentinc#4 rd_kafka_stats_emit_tmr_cb build_docker/./contrib/librdkafka/src/rdkafka.c:1898:2 (clickhouse+0x1e395c8b)
        confluentinc#5 rd_kafka_timers_run build_docker/./contrib/librdkafka/src/rdkafka_timer.c:288:4 (clickhouse+0x1e46498a) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
        confluentinc#6 rd_kafka_thread_main build_docker/./contrib/librdkafka/src/rdkafka.c:2021:3 (clickhouse+0x1e3919e9) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
        confluentinc#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)
        confluentinc#1 rd_kafka_broker_timeout_scan build_docker/./contrib/librdkafka/src/rdkafka_broker.c:880:25 (clickhouse+0x1e37d71d)
        confluentinc#2 rd_kafka_broker_ops_io_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:3416:17 (clickhouse+0x1e37d71d)
        confluentinc#3 rd_kafka_broker_consumer_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:4975:17 (clickhouse+0x1e378e5e) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
        confluentinc#4 rd_kafka_broker_serve build_docker/./contrib/librdkafka/src/rdkafka_broker.c:5080:17 (clickhouse+0x1e378e5e)
        confluentinc#5 rd_kafka_broker_thread_main build_docker/./contrib/librdkafka/src/rdkafka_broker.c:5237:25 (clickhouse+0x1e372619) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)
        confluentinc#6 _thrd_wrapper_function build_docker/./contrib/librdkafka/src/tinycthread.c:576:9 (clickhouse+0x1e47a57b) (BuildId: 7122171f6a93acda7ea89a6d10cce3ad580a715d)

Refs: ClickHouse/ClickHouse#60443
Refs: ClickHouse/ClickHouse#60939
Refs: confluentinc#4625
@rschu1ze
Copy link
Copy Markdown
Member

rschu1ze commented Jul 9, 2024

@antaljanosbenjamin

@azat
Copy link
Copy Markdown
Member Author

azat commented Jul 10, 2024

@antaljanosbenjamin can you merge it? I want to update submodule to point to merge commit. Or it waits something?

@antaljanosbenjamin antaljanosbenjamin merged commit 39d4ed4 into ClickHouse:master Jul 10, 2024
@azat azat deleted the ch/fix-data-race-in-stat branch July 10, 2024 14:28
azat added a commit to azat/ClickHouse that referenced this pull request Aug 8, 2024
The problem here is that ignorelist did not work by some reason, if I
will look at the ignored functions it should not contain any TSan
interseption code, while it does:

    $ lldb-13 clickhouse
    (lldb) target create "clickhouse"
    disas -n rd_avg_rollover
    Current executable set to '/home/azat/ch/tmp/tsan-test/clickhouse' (x86_64).
    (lldb) disas -n rd_avg_rollover
    clickhouse`rd_kafka_stats_emit_avg:
    clickhouse[0x1cbf84a7] <+39>:  leaq   0x30(%r15), %r12
    clickhouse[0x1cbf84ab] <+43>:  movq   %r12, %rdi
    clickhouse[0x1cbf84ae] <+46>:  callq  0x1ccdad40                ; rdk_thread_mutex_lock at tinycthread.c:111
    clickhouse[0x1cbf84b3] <+51>:  leaq   0x58(%r15), %rdi
    clickhouse[0x1cbf84b7] <+55>:  callq  0x71b5390                 ; __tsan_read4
    clickhouse[0x1cbf84bc] <+60>:  cmpl   $0x0, 0x58(%r15)
    clickhouse[0x1cbf84c1] <+65>:  je     0x1cbf8595                ; <+277> [inlined] rd_avg_rollover + 238 at rdavg.h
    clickhouse[0x1cbf84c7] <+71>:  leaq   -0xc8(%rbp), %rdi
    clickhouse[0x1cbf84ce] <+78>:  xorl   %esi, %esi
    clickhouse[0x1cbf84d0] <+80>:  callq  0x1ccdac80                ; rdk_thread_mutex_init at tinycthread.c:62
    clickhouse[0x1cbf84d5] <+85>:  leaq   0x5c(%r15), %rdi
    clickhouse[0x1cbf84d9] <+89>:  callq  0x71b5390                 ; __tsan_read4

    (lldb) disas -n rd_avg_calc
    clickhouse`rd_kafka_broker_ops_io_serve:
    clickhouse[0x1cbdf086] <+1990>: leaq   0x5a4(%rbx), %rdi
    clickhouse[0x1cbdf08d] <+1997>: callq  0x71b5390                 ; __tsan_read4
    clickhouse[0x1cbdf092] <+2002>: cmpl   $0x0, 0x5a4(%rbx)
    clickhouse[0x1cbdf099] <+2009>: je     0x1cbdf12b                ; <+2155> [inlined] rd_kafka_broker_timeout_scan + 719 at rdkafka_broker.c

I guess the reason is that they had been inlined

So now rd_avg_calc() guarded with a mutex.

Refs: ClickHouse/librdkafka#11
Fixes: ClickHouse#60939
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
antaljanosbenjamin added a commit that referenced this pull request Sep 12, 2024
antaljanosbenjamin added a commit that referenced this pull request Sep 17, 2024
antaljanosbenjamin added a commit that referenced this pull request Sep 17, 2024
antaljanosbenjamin added a commit that referenced this pull request Sep 17, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants