ClickHouse: Some queries does not respect max_threads and can make server non-resposible by starting too many threads
Happens on official build.
Repro (tested on 23.7.4.5).
cat <<'EOF' | clickhouse-client -mn
DROP DATABASE IF EXISTS test_max_threads_leak;
CREATE DATABASE test_max_threads_leak;
CREATE TABLE test_max_threads_leak.test
(
`a` String,
`b` LowCardinality(String),
`c` DateTime64(3),
`val` String,
)
ENGINE = MergeTree
ORDER BY (a, b, c)
SETTINGS index_granularity = 8192, parts_to_delay_insert=0;
CREATE TABLE test_max_threads_leak.test_distributed
(
`a` String,
`b` LowCardinality(String),
`c` DateTime64(3),
`val` String,
)
ENGINE = Distributed('default', 'test_max_threads_leak', 'test', rand());
-- creating lot of parts, because the number of the threads clickhouse will start is proportional to the number of parts
SYSTEM STOP MERGES test_max_threads_leak.test;
INSERT INTO test_max_threads_leak.test (a, b, c, val) SELECT 'foo' as a, 'bar' as b, _CAST('1683504000', 'DateTime64') as c, 'baz' as val FROM numbers(4000) SETTINGS max_block_size = 1, min_insert_block_size_bytes=1,min_insert_block_size_rows=1;
select count() from system.parts where table = 'test' and active;
EOF
cat <<'EOF' | clickhouse-benchmark -i 250 -c 25 --max_threads=5
SELECT DISTINCT 'val' AS fieldType, val AS value FROM test_max_threads_leak.test_distributed WHERE (a = 'foo') AND (value != '') AND (positionCaseInsensitiveUTF8(value, 'baz') > 0) AND (b = 'bar') AND (c >= _CAST('1683504000', 'DateTime64')) ORDER BY c DESC LIMIT 5
EOF
clickhouse benchmark here starts 25 concurrent queries with max_threads=5. It should not use more than 125 threads. In practice, it tries to start more than 10000 and crashes with exceptions like
An error occurred while processing the query ...
Code: 439. DB::Exception: Received from localhost:9000. DB::Exception: Cannot schedule a task: cannot allocate thread (threads=238, jobs=238). Stack trace:
0. DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000e91fc37 in /usr/bin/clickhouse
1. ? @ 0x000000000ea0cf46 in /usr/bin/clickhouse
2. ? @ 0x000000000ea0dc8e in /usr/bin/clickhouse
3. ? @ 0x000000000ea08546 in /usr/bin/clickhouse
4. DB::PipelineExecutor::spawnThreads() @ 0x00000000152d58f7 in /usr/bin/clickhouse
5. DB::PipelineExecutor::executeImpl(unsigned long) @ 0x00000000152d447e in /usr/bin/clickhouse
6. DB::PipelineExecutor::execute(unsigned long) @ 0x00000000152d4151 in /usr/bin/clickhouse
7. ? @ 0x00000000152e1f8d in /usr/bin/clickhouse
8. ThreadPoolImpl<std::thread>::worker(std::__list_iterator<std::thread, void*>) @ 0x000000000ea070cf in /usr/bin/clickhouse
9. ? @ 0x000000000ea0d0c1 in /usr/bin/clickhouse
10. ? @ 0x00007fc84a0cc609 in ?
11. __clone @ 0x00007fc849ff1133 in ?
. (CANNOT_SCHEDULE_TASK) (version 23.7.4.5 (official build))
Also, the numbers reported in that error message are incorrect. Older versions (23.3) report it better:
Code: 439. DB::Exception: Received from localhost:9000. DB::Exception: Cannot schedule a task: no free thread (timeout=0) (threads=10000, jobs=10000).
The real threads usage can be checked by query
select event_time, CurrentMetric_GlobalThreadActive from system.metric_log where event_time > now() - 600 order by event_time ;
Was observed after the upgrade to 23.3.
About this issue
- Original URL
- State: closed
- Created a year ago
- Comments: 18 (16 by maintainers)
Commits related to this issue
- Disable logic max_threads=max_distributed_connections when async_socket_for_remote=1 See #53287 — committed to filimonov/ClickHouse by filimonov 10 months ago
So i think we can do 2 things:
patch which can be ported to older versions like 23.3 / 22.8 - i’ve drafted PR https://github.com/ClickHouse/ClickHouse/pull/53504/files
(maybe) get rid of async_socket_for_remote=0 at all?
Yes, the problem is clear, but it is not that easy to fix (except for ignoring this path under
async_socket_for_remote)@filimonov This is likely because now PipelineExecutor uses ThreadPool - #48146
@jorisgio More or less - yes. And actually
max_distributed_connectionsdoes nothing withasync_socket_for_remote.See https://github.com/ClickHouse/ClickHouse/pull/17848
Thanks for the explanation, now I’ve got it. And I’m sorry for my impatience 😃
Additionally, I was able to confirm that: lowering
max_distribured_connectionsallows to workaround the problem.Actually that prefer_localhost_replica optimization quite often leads to some unexpected behaviour (see https://github.com/ClickHouse/ClickHouse/issues/6349 https://github.com/ClickHouse/ClickHouse/issues/37090 https://github.com/ClickHouse/ClickHouse/issues/44487 etc)
Also, i’ve tried to follow the code and found that comment, which (probably) was the original idea: https://github.com/ClickHouse/ClickHouse/blob/dbe13e30168b75d9eaa073228c5242d6ca375337/src/Interpreters/InterpreterSelectQuery.cpp#L2519
I am pretty concinced I talk about the exact same problem, I apologise if am wrong though.
Edit : just to be clear, I believe the problem is that when a query reads from a remote storage, the thread limit in executor is raised to max_distribured_connections instead of being set to max_thread. Which in turn is compound by the fact this setting happens without taking into account optimisation of prefer_localhost_replica,bevause the part of the code that sets it does not take into account the optimisation. And on top of that I believe the code that raises the thread limit is outdated and unnecessary because remote read now uses async io by default, and it does not improve performance as it used to when remote read was blocking. So yeah it does not happen when reading from local table because it does not trigger the executor thread limit increase bevause local read is not a read from remote. But combining prefer_localhost_replica and distributed tabke it uses the full parallelism on the pipeline limited by max_distribured_connections settings, even if the repro is fully local and in fact does not create any connection, and the pipeline can be extremely wide when you create a lot of parts.
It comes from the freshly installed clickhouse, from default config https://github.com/ClickHouse/ClickHouse/blob/9e7e58181360e3b24c27c67241304b2458832973/programs/server/config.xml#L808
Please also try the one I’ve sent.
It will not. Try the test.
Problem does not happen when you query local table. Try the test.
One node is used. No connections to other nodes at all.
Please run the test. The exact one in the first comment. You talk about some different problems.