Skip to content

Create consumers for Kafka tables on fly (but keep them for some period since last used)#57829

Merged
alexey-milovidov merged 11 commits intoClickHouse:masterfrom
azat:kafka-fix-stat-leak
Dec 27, 2023
Merged

Create consumers for Kafka tables on fly (but keep them for some period since last used)#57829
alexey-milovidov merged 11 commits intoClickHouse:masterfrom
azat:kafka-fix-stat-leak

Conversation

@azat
Copy link
Member

@azat azat commented Dec 13, 2023

Changelog category (leave one):

  • Bug Fix (user-visible misbehavior in an official stable release)

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.

Requires: #57822 (not requires, but just better to merge it after it, since in that case I will enable statistics by default here again, or maybe it should be done separately...)
Fixes: #50999 (cc @ilejn)

Note: this should not be backported

@robot-clickhouse
Copy link
Member

robot-clickhouse commented Dec 13, 2023

This is an automated comment for commit ebad1bf 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
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
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
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✅ 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
Integration testsThe integration tests report. In parenthesis the package type is given, and in square brackets are the optional part/total tests❌ failure
Stateless testsRuns stateless functional tests for ClickHouse binaries built in various configurations -- release, debug, with sanitizers, etc❌ failure

@azat azat force-pushed the kafka-fix-stat-leak branch from d8d600c to 8a145e4 Compare December 13, 2023 20:33
@robot-ch-test-poll2 robot-ch-test-poll2 added the pr-bugfix Pull request with bugfix, not backported by default label Dec 13, 2023
@azat azat force-pushed the kafka-fix-stat-leak branch from 8a145e4 to c3a8eb6 Compare December 14, 2023 11:15
@filimonov
Copy link
Contributor

Can a similar problem happen in the case when readings from the topic occur
but statistics_interval_ms is significantly smaller than the poll frequency
(i.e., the queue with messages containing statistics will fill up faster than it will be polled)?

Ideally, it would be better to have some backpressure mechanism, or we can just forbid
having statistics_interval_ms smaller than kafka_flush_interval_ms.

Additionally, it seems that 1 minute to remove a consumer from the pool is too fast. I have seen
cases where transferring data from Kafka to Matview took more than a minute. I think
5-10 minutes of inactivity would be better.

It is also worth mentioning that with that PR, after deleting a consumer,
the information contained in system.kafka_consumers will also disappear (which
makes the table quite inconvenient to use).

Maybe we can try deleting only the internal object (cppkafka::Consumer) while
leaving KafkaConsumer in the pool?

@azat
Copy link
Member Author

azat commented Dec 15, 2023

Can a similar problem happen in the case when readings from the topic occur
but statistics_interval_ms is significantly smaller than the poll frequency

Poll frequency is small enough - 500ms
It does not make sense to use smaller statistics_interval_ms

Also note, that this messages have higher priority AFAIR, so even in case of errors while reading from the topic they will be handled first.

Ideally, it would be better to have some backpressure mechanism, or we can just forbid
having statistics_interval_ms smaller than kafka_flush_interval_ms.

This setting means different, it is how much time polling from kafka can be done without yield.

Additionally, it seems that 1 minute to remove a consumer from the pool is too fast. I have seen
cases where transferring data from Kafka to Matview took more than a minute. I think
5-10 minutes of inactivity would be better.

This 1min timeout is the TTL for inactive consumers, if the data transfer in progress then it is in use and won't be deleted, and even when transfer will be finished it won't be deleted either, and since the default poll frequency 500ms this should be OK. The only reason for keeping it multiple times bigger the 500ms is to preserve the consumers for SELECTs

It is also worth mentioning that with that PR, after deleting a consumer,
the information contained in system.kafka_consumers will also disappear (which
makes the table quite inconvenient to use).

Indeed.

Maybe we can try deleting only the internal object (cppkafka::Consumer) while
leaving KafkaConsumer in the pool?

Yes, this make sense, will update the patch.

@filimonov
Copy link
Contributor

filimonov commented Dec 15, 2023

This 1min timeout is the TTL for inactive consumers, if the data transfer in progress then it is in use and won't be deleted, and even when transfer will be finished it won't be deleted either, and since the default poll frequency 500ms this should be OK. The only reason for keeping it multiple times bigger the 500ms is to preserve the consumers for SELECTs

Creating / removing the consumer is quite expensive as it triggers consumer group rebalance (which is 'stop the world' thing in Kafka).
Sometimes people do wierd things, like staring a lot of kafka table / or a lot of consumers without increasing the thread pool which can work with them.

It's actually a bad thing (BTW - maybe we can create some warning about that? Or inform user someway? Or just make the size of that pool dynamic?), and here delays sometimes can get bigger (no free thread in the pool, nobody can execute StorageKafka::threadFunc so the consumer objects will be created / removed causing rebalances and everythig will become event worse (BTW - it seem like in that way the cleanConsumers(); will also not be called)

@azat azat marked this pull request as draft December 15, 2023 12:01
@azat
Copy link
Member Author

azat commented Dec 15, 2023

BTW - it seem like in that way the cleanConsumers(); will also not be called

Indeed, this is also a problem.

And also there are some other issues, like RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE works only for rd_kafka_destroy_flags, but not for rd_kafka_consumer_close, and this leads to very hangs for closing the consumers, which will also cause thread starvation (UPD: looks like this was a problem of leaving librdkafka objects in incorrect state, but this is not related to this PR, it was my local patch)

So yes, all of this, should be addressed first.

@azat azat changed the title Create consumers for Kafka tables on fly (but keep them for 1min since last used) Create consumers for Kafka tables on fly (but keep them for some period since last used) Dec 15, 2023
@azat azat force-pushed the kafka-fix-stat-leak branch 3 times, most recently from 2345903 to 7265068 Compare December 17, 2023 08:11
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not very obvious choice of moment of setting last_used time.

Why not inside KafkaConsumer::consume? (it's even have last poll time already)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

last_poll_time is not the same, it can differs from the last_used_usec, because, after poll had been finished other things could be done:

  • reading from this source could take some time
  • commit
  • writing this data to MV could take some time

And updating it in notInUse looks like the most logical, since KafkaConsumer should know nothing about TTL (but to avoid tracking this timestamps in a separate map, this time had been added to the KafkaConsumer)

Copy link
Contributor

@filimonov filimonov Dec 29, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe the option to convert consumers from StorageKafka into class with methods pop / push, also maintaining that in_use flag & time of the last usage there would be less confusing (since KafkaConsumer class don't do anything with that anyway). But OK.

@azat
Copy link
Member Author

azat commented Dec 27, 2023

CI is unrelated:

PullRequestCI / BuilderBinClangTidy / Build-binary_tidy (pull_request) Failing after 1m

Dec 21 14:43:50 /build/src/Functions/FunctionsStringHashFixedString.cpp:175:9: error: 'auto md_ctx' can be declared as 'auto *md_ctx' [readability-qualified-auto,-warnings-as-errors]
Dec 21 14:43:50   175 |         auto md_ctx = EVP_MD_CTX_create();
Dec 21 14:43:50       |         ^~~~
Dec 21 14:43:50       |         auto *
Dec 21 14:43:50 24115 warnings generated.

Already fixed in upstream

Stateless tests (release, s3 storage) [1/2] — fail: 1, passed: 2932, skipped: 31

Upgrade check (debug) — check_status.tsv doesn't exists
Upgrade check (tsan) — check_status.tsv doesn't exists

2023.12.21 16:24:13.665560 [ 277732 ] {} <Error> Application: Caught exception while loading metadata: Code: 695. DB::Exception: Load job 'load table test_19.test' failed: Code: 36. DB::Exception: Wrong number of engine arguments.: Cannot attach table `test_19`.`test` from metadata file /var/lib/clickhouse/store/3d1/3d149722-0d54-46dd-b98e-55a40a9ce094/test.sql from query ATTACH TABLE test_19.test UUID 'a596ba43-a355-4a47-b8ff-62be9c2d8df3' (`uid` String, `version` UInt32, `is_deleted` UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) ORDER BY uid SETTINGS vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0, min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1, index_granularity = 5131, ratio_of_defaults_for_sparse_serialization = 0.2101200670003891, replace_long_file_name_to_hash = 1, max_file_name_length = 0, merge_max_block_size = 8602, prefer_fetch_merged_part_size_threshold = 10737418240, min_merge_bytes_to_use_direct_io = 9725563422, index_granularity_bytes = 18086344, allow_vertical_merges_from_compact_to_wide_parts = 1, marks_compress_block_size = 62984, primary_key_compress_block_size = 94732. (BAD_ARGUMENTS), Stack trace (when copying this message, always include the lines below):

AST fuzzer (msan) — SUMMARY: MemorySanitizer: use-of-uninitialized-value (/workspace/clickhouse+0xd7a0259) (BuildId: 4cfde472118d009833a13c8c52e02c2862558e41...

Not enough info, even in core dump (sigh), core dumps should be fixed.

azat added 11 commits December 27, 2023 09:49
…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 <[email protected]>
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 <[email protected]>
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 <[email protected]>
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 <[email protected]>
@azat azat force-pushed the kafka-fix-stat-leak branch from d2865ae to ebad1bf Compare December 27, 2023 08:49
@alexey-milovidov alexey-milovidov self-assigned this Dec 27, 2023
@alexey-milovidov alexey-milovidov merged commit 1a93fd7 into ClickHouse:master Dec 27, 2023
@azat azat deleted the kafka-fix-stat-leak branch December 27, 2023 14:39
@Algunenano
Copy link
Member

https://s3.amazonaws.com/clickhouse-test-reports/0/745d9bb47f3425e28e5660ed7c730038ffece4ee/integration_tests__asan__analyzer__[6_6]/integration_run_parallel4_0.log looks like it might be related

E           Exception: Sanitizer assert found for instance �=================================================================
E           ==1==ERROR: AddressSanitizer: heap-use-after-free on address 0x5250006a4100 at pc 0x55d4ed46d2e2 bp 0x7f7e33b40190 sp 0x7f7e33b3f950
E           WRITE of size 5390 at 0x5250006a4100 thread T2 (TCPHandler)
E               #0 0x55d4ed46d2e1 in __asan_memmove (/usr/bin/clickhouse+0xa9162e1) (BuildId: 74ec4a14a5109c41de109e82d56d8d863845144d)
E               #1 0x55d524744c13 in std::__1::pair<char const*, char*> std::__1::__copy_impl[abi:v15000]<char const, char, void>(char const*, char const*, char*) build_docker/./contrib/llvm-project/libcxx/include/__algorithm/copy.h:56:5
E               #2 0x55d524744c13 in std::__1::pair<char const*, char*> std::__1::__copy[abi:v15000]<char const*, char const*, char*, 0>(char const*, char const*, char*) build_docker/./contrib/llvm-project/libcxx/include/__algorithm/copy.h:94:18
E               #3 0x55d524744c13 in char* std::__1::copy[abi:v15000]<char const*, char*>(char const*, char const*, char*) build_docker/./contrib/llvm-project/libcxx/include/__algorithm/copy.h:103:10
E               #4 0x55d524744c13 in std::__1::enable_if<__is_cpp17_random_access_iterator<char const*>::value, char*>::type std::__1::copy_n[abi:v15000]<char const*, unsigned long, char*>(char const*, unsigned long, char*) build_docker/./contrib/llvm-project/libcxx/include/__algorithm/copy_n.h:61:12
E               #5 0x55d524744c13 in std::__1::char_traits<char>::copy(char*, char const*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/__string/char_traits.h:233:9
E               #6 0x55d524744c13 in std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>& std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>::__assign_no_alias<false>(char const*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/string:2416:5
E               #7 0x55d50eba9497 in std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>::operator=(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) build_docker/./contrib/llvm-project/libcxx/include/string:2506:14
E               #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
E               #9 0x55d51e0ef34d in std::__1::__function::__policy_func<void (cppkafka::KafkaHandleBase&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&)>::operator()[abi:v15000](cppkafka::KafkaHandleBase&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:848:16
E               #10 0x55d51e0ef34d in std::__1::function<void (cppkafka::KafkaHandleBase&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&)>::operator()(cppkafka::KafkaHandleBase&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:1187:12
E               #11 0x55d51e0ef34d in void cppkafka::CallbackInvoker<std::__1::function<void (cppkafka::KafkaHandleBase&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&)>>::operator()<cppkafka::KafkaHandleBase&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>>(cppkafka::KafkaHandleBase&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>&&) const build_docker/./contrib/cppkafka/include/cppkafka/detail/callback_invoker.h:84:24
E               #12 0x55d51e0eebfe in cppkafka::stats_callback_proxy(rd_kafka_s*, char*, unsigned long, void*) build_docker/./contrib/cppkafka/src/configuration.cpp:92:5
E               #13 0x55d51e151e3d in rd_kafka_poll_cb build_docker/./contrib/librdkafka/src/rdkafka.c:3790:7
E               #14 0x55d51e15531b in rd_kafka_consumer_close build_docker/./contrib/librdkafka/src/rdkafka.c:3200:31
E               #15 0x55d51e0f3241 in cppkafka::Consumer::close() build_docker/./contrib/cppkafka/src/consumer.cpp:293:33
E               #16 0x55d51e0f3241 in cppkafka::Consumer::~Consumer() build_docker/./contrib/cppkafka/src/consumer.cpp:82:9
E               #17 0x55d50eb8d12e in std::__1::__shared_count::__release_shared[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:174:9
E               #18 0x55d50eb8d12e in std::__1::__shared_weak_count::__release_shared[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:215:27
E               #19 0x55d50eb8d12e in std::__1::shared_ptr<cppkafka::Consumer>::~shared_ptr[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:702:23
E               #20 0x55d50eb8d12e in DB::KafkaConsumer::~KafkaConsumer() build_docker/./src/Storages/Kafka/KafkaConsumer.cpp:179:1
E               #21 0x55d50ebc96cc in std::__1::__shared_count::__release_shared[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:174:9
E               #22 0x55d50ebc96cc in std::__1::__shared_weak_count::__release_shared[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:215:27
E               #23 0x55d50ebc96cc in std::__1::shared_ptr<DB::KafkaConsumer>::~shared_ptr[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:702:23
E               #24 0x55d50ebc96cc in void std::__1::__destroy_at[abi:v15000]<std::__1::shared_ptr<DB::KafkaConsumer>, 0>(std::__1::shared_ptr<DB::KafkaConsumer>*) build_docker/./contrib/llvm-project/libcxx/include/__memory/construct_at.h:63:13
E               #25 0x55d50ebc96cc in void std::__1::destroy_at[abi:v15000]<std::__1::shared_ptr<DB::KafkaConsumer>, 0>(std::__1::shared_ptr<DB::KafkaConsumer>*) build_docker/./contrib/llvm-project/libcxx/include/__memory/construct_at.h:88:5
E               #26 0x55d50ebc96cc in void std::__1::allocator_traits<std::__1::allocator<std::__1::shared_ptr<DB::KafkaConsumer>>>::destroy[abi:v15000]<std::__1::shared_ptr<DB::KafkaConsumer>, void, void>(std::__1::allocator<std::__1::shared_ptr<DB::KafkaConsumer>>&, std::__1::shared_ptr<DB::KafkaConsumer>*) build_docker/./contrib/llvm-project/libcxx/include/__memory/allocator_traits.h:317:9
E               #27 0x55d50ebc96cc in std::__1::vector<std::__1::shared_ptr<DB::KafkaConsumer>, std::__1::allocator<std::__1::shared_ptr<DB::KafkaConsumer>>>::__base_destruct_at_end[abi:v15000](std::__1::shared_ptr<DB::KafkaConsumer>*) build_docker/./contrib/llvm-project/libcxx/include/vector:833:9
E               #28 0x55d50ebc96cc in std::__1::vector<std::__1::shared_ptr<DB::KafkaConsumer>, std::__1::allocator<std::__1::shared_ptr<DB::KafkaConsumer>>>::__clear[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/vector:827:29
E               #29 0x55d50ebc96cc in std::__1::vector<std::__1::shared_ptr<DB::KafkaConsumer>, std::__1::allocator<std::__1::shared_ptr<DB::KafkaConsumer>>>::clear[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/vector:637:9
E               #30 0x55d50ebc96cc in DB::StorageKafka::shutdown(bool) build_docker/./src/Storages/Kafka/StorageKafka.cpp:471:19
E               #31 0x55d5128bd4ff in DB::IStorage::flushAndShutdown(bool) build_docker/./src/Storages/IStorage.h:573:9
E               #32 0x55d5128bd4ff in DB::InterpreterDropQuery::executeToTableImpl(std::__1::shared_ptr<DB::Context const>, DB::ASTDropQuery&, std::__1::shared_ptr<DB::IDatabase>&, StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>&) build_docker/./src/Interpreters/InterpreterDropQuery.cpp:270:20
E               #33 0x55d5128c45b6 in DB::InterpreterDropQuery::executeToDatabaseImpl(DB::ASTDropQuery const&, std::__1::shared_ptr<DB::IDatabase>&, std::__1::vector<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>, std::__1::allocator<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>>>&) build_docker/./src/Interpreters/InterpreterDropQuery.cpp:415:21
E               #34 0x55d5128ba2aa in DB::InterpreterDropQuery::executeToDatabase(DB::ASTDropQuery const&) build_docker/./src/Interpreters/InterpreterDropQuery.cpp:333:15
E               #35 0x55d5128b6dee in DB::InterpreterDropQuery::execute() build_docker/./src/Interpreters/InterpreterDropQuery.cpp:79:16
E               #36 0x55d513993985 in DB::executeQueryImpl(char const*, char const*, std::__1::shared_ptr<DB::Context>, DB::QueryFlags, DB::QueryProcessingStage::Enum, DB::ReadBuffer*) build_docker/./src/Interpreters/executeQuery.cpp:1108:40
E               #37 0x55d51398a089 in DB::executeQuery(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&, std::__1::shared_ptr<DB::Context>, DB::QueryFlags, DB::QueryProcessingStage::Enum) build_docker/./src/Interpreters/executeQuery.cpp:1286:26
E               #38 0x55d5169a0bc9 in DB::TCPHandler::runImpl() build_docker/./src/Server/TCPHandler.cpp:498:54
E               #39 0x55d5169e644f in DB::TCPHandler::run() build_docker/./src/Server/TCPHandler.cpp:2292:9
E               #40 0x55d51e7e2f8e in Poco::Net::TCPServerConnection::start() build_docker/./base/poco/Net/src/TCPServerConnection.cpp:43:3
E               #41 0x55d51e7e3cb6 in Poco::Net::TCPServerDispatcher::run() build_docker/./base/poco/Net/src/TCPServerDispatcher.cpp:115:20
E               #42 0x55d51ebf141b in Poco::PooledThread::run() build_docker/./base/poco/Foundation/src/ThreadPool.cpp:188:14
E               #43 0x55d51ebea3e7 in Poco::ThreadImpl::runnableEntry(void*) build_docker/./base/poco/Foundation/src/Thread_POSIX.cpp:335:27
E               #44 0x55d4ed46ca1a in asan_thread_start(void*) crtstuff.c
E               #45 0x7f7f37c70ac2  (/lib/x86_64-linux-gnu/libc.so.6+0x94ac2) (BuildId: 203de0ae33b53fee1578b117cb4123e85d0534f0)
E               #46 0x7f7f37d0265f  (/lib/x86_64-linux-gnu/libc.so.6+0x12665f) (BuildId: 203de0ae33b53fee1578b117cb4123e85d0534f0)
E           

@alexey-milovidov
Copy link
Member

Thank you! We overlooked that. @azat, could you please resubmit?

@azat azat restored the kafka-fix-stat-leak branch December 28, 2023 14:32
@azat azat deleted the kafka-fix-stat-leak branch December 28, 2023 14:46
@azat
Copy link
Member Author

azat commented Dec 28, 2023

See #58310

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.

6 participants