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 2ff0bfb0a1)
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 e7592c140e)
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>
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>
That way we can avoid superior timeouts during reading from
system.stack_trace.
Two known cases of such threads are:
- rdk: -- librdkafka threads
- iou-wrk -- io_uring threads
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
Some third-party libraries (i.e. librdkafka) could block it, and in this
case system.stack_trace will return stacktrace for the main process
(usually, basically it could be any thread with non blocked signal).
By replacing sigqueue() with more precise rt_tgsigqueueinfo(), other
threads will not respond to the signal.
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 106042cf41)
Some third-party libraries (i.e. librdkafka) could block it, and in this
case system.stack_trace will return stacktrace for the main process
(usually, basically it could be any thread with non blocked signal).
By replacing sigqueue() with more precise rt_tgsigqueueinfo(), other
threads will not respond to the signal.
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>