Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

DB::Exception: Cannot schedule a task #6833

Closed
ghost opened this issue Sep 5, 2019 · 11 comments
Closed

DB::Exception: Cannot schedule a task #6833

ghost opened this issue Sep 5, 2019 · 11 comments
Labels
bug Confirmed user-visible misbehaviour in official release st-need-info We need extra data to continue (waiting for response)

Comments

@ghost
Copy link

ghost commented Sep 5, 2019

Describe the bug or unexpected behaviour
While running a 3 node clickhouse cluster with replicated tables, we saw a large number of the included stack trace. It seems that we reached a resource/config limit, but it's not clear which one (not cpu, memory or disk). There did appear to be a large number of inter cluster node tcp connections, and a significant number of open files (~3 million per cluster node).

Mostly just looking to see if there is an explanation for runaway tcp connections, and what can be expected in terms of inter node connections/communications.

We did see a large number of waiting tcp connections to each node in the cluster (~20k).

How to reproduce

  • Which ClickHouse server version to use
    19.13.2.19

  • Which interface to use, if matters
    HTTP

  • Non-default settings, if any
    The uncompressed cache is enabled as the queries typically return a small result set (couple hundred rows).

  • Queries to run that lead to unexpected result

select data from table where date = '2019-09-04'

NOTE: These queries target a distributed table.

Also the queried table contains about 220 million rows.

Expected behavior
A clear and concise description of what you expected to happen.

Error message and/or stacktrace

 (version 19.13.2.19)
2019.09.05 12:24:19.973783 [ 3189 ] {45412c11-a1ac-4e24-a9e8-e2bff828d1c1} <Error> executeQuery: Code: 439, e.displayText() = DB::Exception: Cannot schedule a task (version 19.13.2.19) (from 10.2.0.119:29106) (in query: select data from table where date = '2019-09-04'), Stack trace:

0. clickhouse-server(StackTrace::StackTrace()+0x30) [0x6f28600]
1. clickhouse-server(DB::Exception::Exception(std::string const&, int)+0x1f) [0x316365f]
2. clickhouse-server(void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, int, std::optional<unsigned long>)::{lambda()#1}::operator()() const+0x7e) [0x318fabe]
3. clickhouse-server(void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, int, std::optional<unsigned long>)+0x562) [0x3190ce2]
4. clickhouse-server(ThreadPoolImpl<std::thread>::scheduleOrThrow(std::function<void ()>, int, unsigned long)+0x4e) [0x3190fae]
5. clickhouse-server(ThreadFromGlobalPool::ThreadFromGlobalPool<void (DB::ParallelInputsProcessor<DB::UnionBlockInputStream::Handler>::*)(std::shared_ptr<DB::ThreadGroupStatus>, unsigned long), DB::ParallelInputsProcessor<DB::UnionBlockInputStream::Handler>*, std::shared_ptr<DB::ThreadGroupStatus>, unsigned long&>(void (DB::ParallelInputsProcessor<DB::UnionBlockInputStream::Handler>::*&&)(std::shared_ptr<DB::ThreadGroupStatus>, unsigned long), DB::ParallelInputsProcessor<DB::UnionBlockInputStream::Handler>*&&, std::shared_ptr<DB::ThreadGroupStatus>&&, unsigned long&)+0x15d) [0x5dbac4d]
6. clickhouse-server(DB::UnionBlockInputStream::readImpl()+0x2a0) [0x5dbb280]
7. clickhouse-server(DB::IBlockInputStream::read()+0x238) [0x5c53178]
8. clickhouse-server(DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::atomic<bool>*)+0x58) [0x5c72fb8]
9. clickhouse-server(DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, DB::Context&, std::function<void (std::string const&)>, std::function<void (std::string const&)>)+0x614) [0x5ebeb44]
10. clickhouse-server(DB::HTTPHandler::processQuery(Poco::Net::HTTPServerRequest&, HTMLForm&, Poco::Net::HTTPServerResponse&, DB::HTTPHandler::Output&)+0x16dc) [0x31b19fc]
11. clickhouse-server(DB::HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest&, Poco::Net::HTTPServerResponse&)+0x443) [0x31b4bb3]
12. clickhouse-server(Poco::Net::HTTPServerConnection::run()+0x2af) [0x6a76f2f]
13. clickhouse-server(Poco::Net::TCPServerConnection::start()+0xf) [0x6a6dc7f]
14. clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x166) [0x6a6e046]
15. clickhouse-server(Poco::PooledThread::run()+0x77) [0x70fd527]
16. clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x38) [0x70f96e8]
17. clickhouse-server() [0x76cb73f]
18. /lib64/libpthread.so.0(+0x7dd5) [0x7ff26c96edd5]
19. /lib64/libc.so.6(clone+0x6d) [0x7ff26c39602d]

Additional context
If the problem is too many tcp connections, would chproxy be a recommended solution?

https://github.com/Vertamedia/chproxy

@ghost ghost added the bug Confirmed user-visible misbehaviour in official release label Sep 5, 2019
@abyss7
Copy link
Contributor

abyss7 commented Sep 6, 2019

Can you provide listing of SHOW PROCESSLIST while receiving these exceptions?

This exception means that you have exceeded the queue size of global thread pool which is 10k.

@abyss7 abyss7 added st-need-info We need extra data to continue (waiting for response) v19.13 labels Sep 6, 2019
@ghost
Copy link
Author

ghost commented Sep 22, 2019

Sorry for the delay on responding to this.

Just got into this state again and show processlist is unable to run since a task cannot be scheduled. After restarting, show processlist shows nothing.

@ghost
Copy link
Author

ghost commented Sep 22, 2019

@abyss7 Is there a way to configure or increase the global thread pool limit?

@alexey-milovidov
Copy link
Member

@markcorwin-iex There is no way to configure this limit right now.

@alexey-milovidov alexey-milovidov added v19.14 and removed st-need-info We need extra data to continue (waiting for response) labels Jan 26, 2020
@alexey-milovidov
Copy link
Member

alexey-milovidov commented Jan 26, 2020

The global limit is 10 000 threads. It should be enough in all but pathological cases.

Probably you are using Merge table on top of huge number of Distributed tables?
In that case, every Distributed table will create multiple connections and every connection will correspond to a thread on remote server.

(You can use Distributed table on top of Merge tables instead).

@alexey-milovidov alexey-milovidov added the st-need-info We need extra data to continue (waiting for response) label Jan 26, 2020
@ghost
Copy link
Author

ghost commented Jan 26, 2020

@alexey-milovidov

https://www.altinity.com/blog/2018/5/10/circular-replication-cluster-topology-in-clickhouse

Following this article, I setup circular replication with 3 shards across a 3 node cluster. Node 1 had shards 1,2, Node 2 had shards 2,3, and Node 3 had shards 3,1. These were all merge tables, and one distributed table on each node pointed to local shards.

@azat
Copy link
Collaborator

azat commented Jan 28, 2020

It also happened to me, and AFAIR the GlobalThreadPool::instance().shutdown was set to true by some reason, and there was nothing in logs that can lead to this (no errors before Cannot schedule a task and any subsequent query fails with this error after).

At that time I cannot find the culprit, once this will be reproduced will try to find something

@azat
Copy link
Collaborator

azat commented Jan 28, 2020

Hm, am I missing something or any unhandled exception from job that is scheduled in thread pool will shutdown it? (And also the exception will not be logged)

@proxo
Copy link

proxo commented Feb 3, 2020

I've the same issue i.e. : Code: 439, e.displayText() = DB::Exception: Cannot schedule a task,
when creating more than 10 000 buffer tables. It looks like that a buffer table allocates and holds a thread from the global thread pool.

How to reproduce

  • Run python script with clickhouse driver client:
for i in range(10002):
        table_name = "tmp_test_table_{}".format(i)        
        client.execute("""create table {table_name} (event_date  Date) ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY event_date""".format(**locals()))
        client.execute("""CREATE TABLE IF NOT EXISTS  buff_{table_name} AS default.{table_name} ENGINE = Buffer(default, {table_name}, 16, 60, 300, 10000, 50000, 10240000, 10240000);""".format(table_name=table_name))
  • At some point CREATE statement fails with an error 'DB::Exception: Cannot schedule a task'

  • select metric, value from system.metrics where metric in ('GlobalThreadActive') returns value around 10k.

azat added a commit to azat/ClickHouse that referenced this issue Mar 15, 2020
Otherwise GlobalThreadPool can be terminated (for example due to an
exception from the ParallelInputsHandler::onFinish/onFinishThread, from
ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread,
since writeToTemporaryFile() can definitelly throw) and the server will
not accept new connections (or/and execute queries) anymore.

Here is possible stacktrace (it is a bit inaccurate, due to
optimizations I guess, and it had been obtained with the
DB::tryLogCurrentException() in the catch block of the
ThreadPoolImpl::worker()):

    2020.02.16 22:30:40.415246 [ 45909 ] {} <Error> ThreadPool: Unhandled exception in the ThreadPool(10000,1000,10000) the loop will be shutted down: Code: 241, e.displayText() = DB::Exception: Memory limit (total) exceeded: would use 279.40 GiB (attempt to allocate chunk of 4205536 bytes), maximum: 279.40 GiB, Stack trace (when copying this message, always include the lines below):

    1.  Common/Exception.cpp:35: DB::Exception::Exception(...)
    ...
    6.  Common/Allocator.h:102: void DB::PODArrayBase<8ul, 4096ul, Allocator<false, false>, 15ul, 16ul>::reserve<>(unsigned long) (.part.0)
    7.  Interpreters/Aggregator.cpp:1040: void DB::Aggregator::writeToTemporaryFileImpl<...>(...)
    8.  Interpreters/Aggregator.cpp:719: DB::Aggregator::writeToTemporaryFile(...)
    9.  include/memory:4206: DB::Aggregator::writeToTemporaryFile(...)
    10. DataStreams/ParallelInputsProcessor.h:223: DB::ParallelInputsProcessor<DB::ParallelAggregatingBlockInputStream::Handler>::thread(...)

Refs: ClickHouse#6833 (comment)
(Reference to particular comment, since I'm not sure about the initial issue)
@azat
Copy link
Collaborator

azat commented Mar 19, 2020

Looks like original issue should be fixed with #9154

Since according to the comment

Following this article, I setup circular replication with 3 shards across a 3 node cluster. Node 1 had shards 1,2, Node 2 had shards 2,3, and Node 3 had shards 3,1. These were all merge tables, and one distributed table on each node pointed to local shards.

It does not looks like too many Buffer/Distributed tables that acquire tons of threads (and there is a comment that restart fixes the issue)

@alexey-milovidov
Copy link
Member

Fixed in master.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Confirmed user-visible misbehaviour in official release st-need-info We need extra data to continue (waiting for response)
Projects
None yet
Development

No branches or pull requests

4 participants