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

Create consumers for Kafka tables on fly with TTL (resubmit) #58310

Merged
merged 15 commits into from Dec 30, 2023

Conversation

azat
Copy link
Collaborator

@azat azat commented Dec 28, 2023

Changelog category (leave one):

  • Improvement

Changelog entry (a user-readable short description of the changes that goes to CHANGELOG.md):

Create consumers for Kafka tables on fly (but keep them for some period - kafka_consumers_pool_ttl_ms, since last used), this should fix problem with statistics for system.kafka_consumers (that does not consumed when nobody reads from Kafka table, which leads to live memory leak and slow table detach) and also this PR enables stats for system.kafka_consumers by default again.

Pool of consumers created a problem for librdkafka internal statistics,
you need to read from the queue always, while in ClickHouse consumers
created regardless are there any readers or not (attached materialized
views or direct SELECTs).

Otherwise, this statistics messages got queued and never released,
which:

  • creates live memory leak
  • and also makes destroy very slow, due to librdkafka internals (it
    moves entries from this queue into another linked list, but in a
    with sorting, which is incredibly slow for linked lists)

So the idea is simple, let's create a pool of consumers only when they
are required, and destroy them after some timeout (right now it is 60
seconds) if nobody uses them, that way this problem should gone.

This should also reduce number of internal librdkafka threads, when
nobody reads from Kafka tables.

Fixes: #50999 (cc @ilejn)
Resubmits: #57829 (the last patch from the PR fixes the use-after-free reported here #57829 (comment))

….size())

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 123d63e)
Fixes: ClickHouse#42777
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 51d4f58)
…e last used)

Pool of consumers created a problem for librdkafka internal statistics,
you need to read from the queue always, while in ClickHouse consumers
created regardless are there any readers or not (attached materialized
views or direct SELECTs).

Otherwise, this statistics messages got queued and never released,
which:
- creates live memory leak
- and also makes destroy very slow, due to librdkafka internals (it
  moves entries from this queue into another linked list, but in a
  with sorting, which is incredibly slow for linked lists)

So the idea is simple, let's create a pool of consumers only when they
are required, and destroy them after some timeout (right now it is 60
seconds) if nobody uses them, that way this problem should gone.

This should also reduce number of internal librdkafka threads, when
nobody reads from Kafka tables.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit e7592c1)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit db74549)
This will make system.kafka_consumers more useful, since after TTL
consumer object will be removed prio this patch, but after, all
information will be preserved.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 2ff0bfb)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit b19b70b)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit a7453f7)
Since pool may exceed threads, while we need to run this thread always
to avoid memory leaking.

And this should not be a problem since librdkafka has multiple threads
for each consumer (5!) anyway.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 06a9e9a)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 1f03a21)
Actually now we can create consumer object in the ctor, no need to do
this in startup(), since consumer now do not connects to kafka.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 0321820)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit ebad1bf)
CI founds [1]:

    Exception: Sanitizer assert found for instance �=================================================================
    ==1==ERROR: AddressSanitizer: heap-use-after-free on address 0x5250006a4100 at pc 0x55d4ed46d2e2 bp 0x7f7e33b40190 sp 0x7f7e33b3f950
    WRITE of size 5390 at 0x5250006a4100 thread T2 (TCPHandler)
       8 0x55d50eba9497 in DB::KafkaConsumer::setRDKafkaStat(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) build_docker/./src/Storages/Kafka/KafkaConsumer.h:117:22
       12 0x55d51e0eebfe in cppkafka::stats_callback_proxy(rd_kafka_s*, char*, unsigned long, void*) build_docker/./contrib/cppkafka/src/configuration.cpp:92:5
       13 0x55d51e151e3d in rd_kafka_poll_cb build_docker/./contrib/librdkafka/src/rdkafka.c:3790:7
       14 0x55d51e15531b in rd_kafka_consumer_close build_docker/./contrib/librdkafka/src/rdkafka.c:3200:31
       15 0x55d51e0f3241 in cppkafka::Consumer::close() build_docker/./contrib/cppkafka/src/consumer.cpp:293:33
       16 0x55d51e0f3241 in cppkafka::Consumer::~Consumer() build_docker/./contrib/cppkafka/src/consumer.cpp:82:9
       20 0x55d50eb8d12e in DB::KafkaConsumer::~KafkaConsumer() build_docker/./src/Storages/Kafka/KafkaConsumer.cpp:179:1

    0x5250006a4100 is located 0 bytes inside of 8736-byte region [0x5250006a4100,0x5250006a6320)
    freed by thread T2 (TCPHandler) here:
       0 0x55d4ed4a26b2 in operator delete(void*, unsigned long) (/usr/bin/clickhouse+0xa94b6b2) (BuildId: 74ec4a14a5109c41de109e82d56d8d863845144d)
       1 0x55d50eb8ca55 in void std::__1::__libcpp_operator_delete[abi:v15000]<void*, unsigned long>(void*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/new:256:3
       2 0x55d50eb8ca55 in void std::__1::__do_deallocate_handle_size[abi:v15000]<>(void*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/new:282:10
       3 0x55d50eb8ca55 in std::__1::__libcpp_deallocate[abi:v15000](void*, unsigned long, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/new:296:14
       4 0x55d50eb8ca55 in std::__1::allocator<char>::deallocate[abi:v15000](char*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/__memory/allocator.h:128:13
       5 0x55d50eb8ca55 in std::__1::allocator_traits<std::__1::allocator<char>>::deallocate[abi:v15000](std::__1::allocator<char>&, char*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/__memory/allocator_traits.h:282:13
       6 0x55d50eb8ca55 in std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>::~basic_string() build_docker/./contrib/llvm-project/libcxx/include/string:2334:9
       7 0x55d50eb8ca55 in DB::KafkaConsumer::~KafkaConsumer() build_docker/./src/Storages/Kafka/KafkaConsumer.cpp:179:1

  [1]: https://s3.amazonaws.com/clickhouse-test-reports/0/745d9bb47f3425e28e5660ed7c730038ffece4ee/integration_tests__asan__analyzer__%5B6_6%5D/integration_run_parallel4_0.log

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
@azat azat force-pushed the kafka-fix-stat-leak-resubmit branch from af6f594 to ecf7188 Compare December 28, 2023 14:48
@robot-clickhouse robot-clickhouse added the pr-bugfix Pull request with bugfix, not backported by default label Dec 28, 2023
@robot-clickhouse
Copy link
Member

robot-clickhouse commented Dec 28, 2023

This is an automated comment for commit 853fdfe with description of existing statuses. It's updated for the latest CI running

❌ Click here to open a full report in a separate page

Successful checks
Check nameDescriptionStatus
AST fuzzerRuns randomly generated queries to catch program errors. The build type is optionally given in parenthesis. If it fails, ask a maintainer for help✅ success
ClickBenchRuns [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table✅ success
ClickHouse build checkBuilds ClickHouse in various configurations for use in further steps. You have to fix the builds that fail. Build logs often has enough information to fix the error, but you might have to reproduce the failure locally. The cmake options can be found in the build log, grepping for cmake. Use these options and follow the general build process✅ success
Compatibility checkChecks that clickhouse binary runs on distributions with old libc versions. If it fails, ask a maintainer for help✅ success
Docker image for serversThe check to build and optionally push the mentioned image to docker hub✅ success
Docs checkThere's no description for the check yet, please add it to tests/ci/ci_config.py:CHECK_DESCRIPTIONS✅ success
Fast testNormally this is the first check that is ran for a PR. It builds ClickHouse and runs most of stateless functional tests, omitting some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described here✅ success
Flaky testsChecks if new added or modified tests are flaky by running them repeatedly, in parallel, with more randomization. Functional tests are run 100 times with address sanitizer, and additional randomization of thread scheduling. Integrational tests are run up to 10 times. If at least once a new test has failed, or was too long, this check will be red. We don't allow flaky tests, read the doc✅ success
Install packagesChecks that the built packages are installable in a clear environment✅ success
Integration testsThe integration tests report. In parenthesis the package type is given, and in square brackets are the optional part/total tests✅ success
Mergeable CheckChecks if all other necessary checks are successful✅ success
Performance ComparisonMeasure changes in query performance. The performance test report is described in detail here. In square brackets are the optional part/total tests✅ success
SQLTestThere's no description for the check yet, please add it to tests/ci/ci_config.py:CHECK_DESCRIPTIONS✅ success
SQLancerFuzzing tests that detect logical bugs with SQLancer tool✅ success
SqllogicRun clickhouse on the sqllogic test set against sqlite and checks that all statements are passed✅ success
Stateful testsRuns stateful functional tests for ClickHouse binaries built in various configurations -- release, debug, with sanitizers, etc✅ success
Stateless testsRuns stateless functional tests for ClickHouse binaries built in various configurations -- release, debug, with sanitizers, etc✅ success
Stress testRuns stateless functional tests concurrently from several clients to detect concurrency-related errors✅ success
Style CheckRuns a set of checks to keep the code style clean. If some of tests failed, see the related log from the report✅ success
Unit testsRuns the unit tests for different release types✅ success
Check nameDescriptionStatus
Bugfix validate checkChecks that either a new test (functional or integration) or there some changed tests that fail with the binary built on master branch❌ error
CI runningA meta-check that indicates the running CI. Normally, it's in success or pending state. The failed status indicates some problems with the PR⏳ pending
Upgrade checkRuns stress tests on server version from last release and then tries to upgrade it to the version from the PR. It checks if the new server can successfully startup without any errors, crashes or sanitizer asserts❌ failure

@alexey-milovidov alexey-milovidov self-assigned this Dec 28, 2023
@azat
Copy link
Collaborator Author

azat commented Dec 29, 2023

AST fuzzer (msan) — Logical error: 'CopyTransform expects more than 1 outputs, got 1'.

Stateless tests (aarch64) — fail: 1, passed: 5940, skipped: 29

  • 00002_log_and_exception_messages_formatting - cc @tavplubix

Stateless tests (debug, s3 storage) [6/6] — Tests are not finished, fail: 9, passed: 620, skipped: 11

We've discussed this with @filimonov and he pointed out that everything
else (except for rdkafka_stat/rdkafka_stat_mutex) is done via members
orders, so let's do it in the same style.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
The callchain of the kafka consumer is very tricky, so for the sake of
common sense let's just clean the messages on moving out consumer (and
in dtor, but this is just to keep that two code path in sync).

(Also reported by @filimonov)

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
@tavplubix
Copy link
Member

00002_log_and_exception_messages_formatting - cc @tavplubix

#58351

@alexey-milovidov alexey-milovidov merged commit 8fc05e2 into ClickHouse:master Dec 30, 2023
261 of 265 checks passed
@azat azat deleted the kafka-fix-stat-leak-resubmit branch December 31, 2023 16:51
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
pr-bugfix Pull request with bugfix, not backported by default
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

5 participants