Commit Graph

314 Commits

Author SHA1 Message Date
Azat Khuzhin
055c231438 Disable system.kafka_consumers by default (due to possible live memory leak)
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>
2023-12-13 18:13:13 +01:00
Alexey Milovidov
82c461031e Fix build 2023-11-13 10:09:23 +01:00
kssenii
f2c0434c4d Merge remote-tracking branch 'origin/master' into minor-improvements-for-s3-queue 2023-11-06 15:51:11 +01:00
kssenii
da21413354 Better shutdown 2023-11-06 15:47:57 +01:00
Kruglov Pavel
5533258091
Merge pull request #55477 from Avogar/streaming-errors
Allow to save unparsed records and errors in RabbitMQ, NATS and FileLog engines
2023-11-01 14:35:35 +01:00
Stas Morozov
5a922014f7
Avoid problem with configuration queued.min.messages automatically for Kafka storage 2023-10-30 13:53:52 +03:00
avogar
1afd655bbc Allow to save unparsed records and errors in RabbitMQ, NATS and FileLog engines 2023-10-10 16:21:33 +00:00
Kruglov Pavel
af8b1e4978
Merge pull request #54855 from Avogar/obsolete-format-settings
Move obsolete format settings to separate section
2023-09-25 18:00:31 +02:00
Robert Schulze
f5137dd0b4
More clang-tidy fixes 2023-09-21 14:40:57 +00:00
avogar
54cee196bd Move obsolete format settings to separate section 2023-09-20 16:00:28 +00:00
Sergei Trifonov
802579f3f1
Merge pull request #49618 from ClickHouse/concurrency-control-controllable
Make concurrency control controllable
2023-08-29 19:44:51 +02:00
Ilya Golshtein
f58b4a812d exceptions_kafka_consumers: code review suggestions are addressed 2023-08-24 13:10:21 +00:00
Ilya Golshtein
901240eede exceptions_kafka_consumers: clang-tidy improvement 2023-08-24 08:44:08 +00:00
Ilya Golshtein
ec0c12960a exceptions_kafka_consumers: stylecheck 2023-08-23 23:10:25 +00:00
Ilya Golshtein
e7213ef5b9 exceptions_kafka_consumers: new test, MV exceptions propagated 2023-08-23 22:46:04 +00:00
Ilya Golshtein
7cd8237a42 exceptions_kafka_consumers: initial 2023-08-22 10:26:52 +00:00
Ilya Golshtein
ade9c3d970 system_kafka_consumers: per consumer librdkafka stat 2023-08-10 22:35:05 +00:00
Ilya Golshtein
978d8bf9a6 system_kafka_consumers: cleanup with exceptions 2023-08-10 22:35:05 +00:00
Ilya Golshtein
7dab2dc041 system_kafka_consumers: exceptions seem to work 2023-08-10 22:35:05 +00:00
Ilya Golshtein
80829a6bf7 system_kafka_consumers: no offset_committed, new test with MV 2023-08-10 22:35:05 +00:00
Ilya Golshtein
6d6aae24f8 system_kafka_consumers: no friendship with StorageKafka 2023-08-10 22:35:05 +00:00
Ilya Golshtein
ae3e6f0fac system_kafka_consumers: getMemberId introduced to avoid leak 2023-08-10 22:35:05 +00:00
Ilya Golshtein
46be5e5782 system_kafka_consumers: cleanup, test should be more stable 2023-08-10 22:35:05 +00:00
Ilya Golshtein
803a80b098 system_kafka_consumers: refactored with getStat, test passed 2023-08-10 22:35:05 +00:00
Ilya Golshtein
a86548a733 system_kafka_consumers: statistics.interval.ms default, tiny cleanup 2023-08-10 22:35:05 +00:00
Ilya Golshtein
dd6727be3e system_kafka_consumers: style check fixes 2023-08-10 22:35:05 +00:00
Ilya Golshtein
bba4dcacc7 system_kafka_consumers: small code cleanup 2023-08-10 22:35:05 +00:00
Ilya Golshtein
b71c2bedfc system_kafka_consumers: rebalance test is ok, statistics_interval_ms 2023-08-10 22:35:05 +00:00
Ilya Golshtein
83566e132a system_kafka_consumers: all_consumers introduced, offsets work 2023-08-10 22:35:05 +00:00
Ilya Golshtein
de49d7e5a0 system_kafka_consumers: minor change 2023-08-10 22:35:05 +00:00
Ilya Golshtein
ae6e0469ad system_kafka_consumers: rdfakfa stat with attempt to use json 2023-08-10 22:35:05 +00:00
Ilya Golshtein
40fd53c3e3 system_kafka_consumers: style check 2023-08-10 22:35:05 +00:00
Ilya Golshtein
88f5dd5db8 system_kafka_consumers: integration tests passed 2023-08-10 22:35:05 +00:00
Ilya Golshtein
957787d96a system_kafka_consumers: kind of works, test added 2023-08-10 22:35:05 +00:00
Ilya Golshtein
36fb330811 system_kafka_consumers: assignments 2023-08-10 22:35:05 +00:00
Sergei Trifonov
01196ac41f
Merge branch 'master' into concurrency-control-controllable 2023-08-01 15:40:50 +02:00
Yury Bogomolov
b2ee4505fb review fixes 2023-07-28 16:06:03 +00:00
Yury Bogomolov
c8b128aad4 add predefined macros support in kafka engine settings 2023-07-21 01:06:49 +04:00
Alexey Milovidov
2c96580a77
Merge branch 'master' into concurrency-control-controllable 2023-07-04 23:16:04 +03:00
Antonio Andelic
b11f744252
Correctly disable async insert with deduplication when it's not needed (#50663)
* Correctly disable async insert when it's not used

* Better

* Add comment

* Better

* Fix tests

---------

Co-authored-by: Nikita Mikhaylov <mikhaylovnikitka@gmail.com>
2023-06-07 20:33:08 +02:00
Alexander Tokmakov
821b64b420 apply review suggestions 2023-05-22 15:18:29 +02:00
AVMusorin
418a61a68c
Allow using Alias column type for KafkaEngine
```
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>
2023-05-15 15:39:58 +02:00
Azat Khuzhin
2f9a7cba8a Fix stack-use-after-scope in KafkaProducer::produce()
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>
2023-05-10 12:15:17 +02:00
Alexey Milovidov
6fddb5bad3 Simplification 2023-05-07 06:31:00 +02:00
Alexey Milovidov
a695d6227d Make concurrency control controllable 2023-05-07 06:16:30 +02:00
Michael Kolupaev
683077890f Highly questionable refactoring (getInputMultistream() nonsense) 2023-04-17 04:58:32 +00:00
kssenii
8537ee400d Fix kafka test 2023-03-20 11:22:25 +01:00
kssenii
bb0beb7449 Merge remote-tracking branch 'upstream/master' into named-collections-finish 2023-03-17 13:02:36 +01:00
Robert Schulze
5b91631722
Merge pull request #47138 from AVMusorin/disallow-default-columns-kafka-engine
Prohibit DEFAULT/EPHEMERAL/ALIAS/MATERIALIZED columns in KafkaEngine
2023-03-07 20:36:46 +01:00
kssenii
8567e3976b Fix kafka 2023-03-06 12:30:25 +01:00