It is not safe to use statistics because of how KafkaEngine works - it
pre-creates consumers, and this leads to the situation when this
statistics entries generated (RD_KAFKA_OP_STATS), but never consumed.
Which creates a live memory leak for a server with Kafka tables, but
without materialized view attached to it (and no SELECT).
Another problem is that this makes shutdown very slow, because of how
pending queue entries are handled in librdkafka, it uses
TAILQ_INSERT_SORTED, which is sorted insert into linked list, which
works incredibly slow (likely you will never wait till it ends and kill
the server)
For instance in my production setup the server was running for ~67 days
with such table, and it got 1'942'233 `TAILQ_INSERT_SORTED` entries
(which perfectly matches by the way - `67*86400/3` = 1'929'600), and it
moved only 289'806 entries for a few hours, though I'm not sure how much
time the process was in the running state, since most of the time it was
with debugger attached.
So for now let's disable it, to make this patch easy for backporting,
and I will think about long term fix - do not pre-create consumers in
Kafka engine.
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
```
create table kafka
(
a UInt32,
a_str String Alias toString(a)
) engine = Kafka;
create table data
(
a UInt32;
a_str String
) engine = MergeTree
order by tuple();
create materialized view data_mv to data
(
a UInt32,
a_str String
) as
select a, a_str from kafka;
```
Alias type works as expected in comparison with MATERIALIZED/EPHEMERAL
or column with default expression.
Ref: https://github.com/ClickHouse/ClickHouse/pull/47138
Co-authored-by: Azat Khuzhin <a3at.mail@gmail.com>
ASan report:
==1==ERROR: AddressSanitizer: stack-use-after-scope on address 0x7f7540186b20 at pc 0x55656ef458ad bp 0x7f754114acf0 sp 0x7f754114a4b8
READ of size 2 at 0x7f7540186b20 thread T2 (TCPHandler)
0 0x55656ef458ac in __asan_memcpy (/usr/bin/clickhouse+0x11b028ac) (BuildId: 6ba63057df561f6dc16fb8fd7623fd858bf327d1)
1 0x55659f62daf8 in rd_kafka_msg_new00 build_docker/./contrib/librdkafka/src/rdkafka_msg.c:192:3
2 0x55659f62daf8 in rd_kafka_msg_new0 build_docker/./contrib/librdkafka/src/rdkafka_msg.c:257:8
3 0x55659f630635 in rd_kafka_producev build_docker/./contrib/librdkafka/src/rdkafka_msg.c:642:23
4 0x55659f4a865c in cppkafka::Producer::do_produce() build_docker/./contrib/cppkafka/src/producer.cpp:128:19
5 0x55659f4a8306 in cppkafka::Producer::produce() build_docker/./contrib/cppkafka/src/producer.cpp:69:5
6 0x5565961e748c in DB::KafkaProducer::produce() build_docker/./src/Storages/Kafka/KafkaProducer.cpp:70:23
7 0x5565962155b2 in DB::MessageQueueSink::consume(DB::Chunk) build_docker/./src/Storages/MessageQueueSink.cpp:64:23
8 0x5565980b9e8e in DB::SinkToStorage::onConsume(DB::Chunk) build_docker/./src/Processors/Sinks/SinkToStorage.cpp:18:5
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>