mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #36855 from ClickHouse/serxa-docs-dev-threads
Add docs for developers on threads and jobs
This commit is contained in:
commit
96b7ff3454
@ -167,6 +167,34 @@ Config is read from multiple files (in XML or YAML format) and merged into singl
|
||||
|
||||
For queries and subsystems other than `Server` config is accessible using `Context::getConfigRef()` method. Every subsystem that is capable of reloading it's config without server restart should register itself in reload callback in `Server::main()` method. Note that if newer config has an error, most subsystems will ignore new config, log warning messages and keep working with previously loaded config. Due to the nature of `AbstractConfiguration` it is not possible to pass reference to specific section, so `String config_prefix` is usually used instead.
|
||||
|
||||
## Threads and jobs {#threads-and-jobs}
|
||||
|
||||
To execute queries and do side activities ClickHouse allocates threads from one of thread pools to avoid frequent thread creation and destruction. There are a few thread pools, which are selected depending on a purpose and structure of a job:
|
||||
* Server pool for incoming client sessions.
|
||||
* Global thread pool for general purpose jobs, background activities and standalone threads.
|
||||
* IO thread pool for jobs that are mostly blocked on some IO and are not CPU-intensive.
|
||||
* Background pools for periodic tasks.
|
||||
* Pools for preemptable tasks that can be split into steps.
|
||||
|
||||
Server pool is a `Poco::ThreadPool` class instance defined in `Server::main()` method. It can have at most `max_connection` threads. Every thread is dedicated to a single active connection.
|
||||
|
||||
Global thread pool is `GlobalThreadPool` singleton class. To allocate thread from it `ThreadFromGlobalPool` is used. It has an interface similar to `std::thread`, but pulls thread from the global pool and does all necessary initializations. It is configured with the following settings:
|
||||
* `max_thread_pool_size` - limit on thread count in pool.
|
||||
* `max_thread_pool_free_size` - limit on idle thread count waiting for new jobs.
|
||||
* `thread_pool_queue_size` - limit on scheduled job count.
|
||||
|
||||
Global pool is universal and all pools described below are implemented on top of it. This can be thought of as a hierarchy of pools. Any specialized pool takes its threads from the global pool using `ThreadPool` class. So the main purpose of any specialized pool is to apply limit on the number of simultaneous jobs and do job scheduling. If there are more jobs scheduled than threads in a pool, `ThreadPool` accumulates jobs in a queue with priorities. Each job has an integer priority. Default priority is zero. All jobs with higher priority values are started before any job with lower priority value. But there is no difference between already executing jobs, thus priority matters only when the pool in overloaded.
|
||||
|
||||
IO thread pool is implemented as a plain `ThreadPool` accessible via `IOThreadPool::get()` method. It is configured in the same way as global pool with `max_io_thread_pool_size`, `max_io_thread_pool_free_size` and `io_thread_pool_queue_size` settings. The main purpose of IO thread pool is to avoid exhaustion of the global pool with IO jobs, which could prevent queries from fully utilizing CPU.
|
||||
|
||||
For periodic task execution there is `BackgroundSchedulePool` class. You can register tasks using `BackgroundSchedulePool::TaskHolder` objects and the pool ensures that no task runs two jobs at the same time. It also allows you to postpone task execution to a specific instant in the future or temporarily deactivate task. Global `Context` provides a few instances of this class for different purposes. For general purpose tasks `Context::getSchedulePool()` is used.
|
||||
|
||||
There are also specialized thread pools for preemptable tasks. Such `IExecutableTask` task can be split into ordered sequence of jobs, called steps. To schedule these tasks in a manner allowing short tasks to be prioritied over long ones `MergeTreeBackgroundExecutor` is used. As name suggests it is used for background MergeTree related operations such as merges, mutations, fetches and moves. Pool instances are available using `Context::getCommonExecutor()` and other similar methods.
|
||||
|
||||
No matter what pool is used for a job, at start `ThreadStatus` instance is created for this job. It encapsulates all per-thread information: thread id, query id, performance counters, resource consumption and many other useful data. Job can access it via thread local pointer by `CurrentThread::get()` call, so we do not need to pass it to every function.
|
||||
|
||||
If thread is related to query execution, then the most important thing attached to `ThreadStatus` is query context `ContextPtr`. Every query has its master thread in the server pool. Master thread does the attachment by holding an `ThreadStatus::QueryScope query_scope(query_context)` object. Master thread also creates a thread group represented with `ThreadGroupStatus` object. Every additional thread that is allocated during this query execution is attached to its thread group by `CurrentThread::attachTo(thread_group)` call. Thread groups are used to aggregate profile event counters and track memory consumption by all threads dedicated to a single task (see `MemoryTracker` and `ProfileEvents::Counters` classes for more information).
|
||||
|
||||
## Distributed Query Execution {#distributed-query-execution}
|
||||
|
||||
Servers in a cluster setup are mostly independent. You can create a `Distributed` table on one or all servers in a cluster. The `Distributed` table does not store data itself – it only provides a “view” to all local tables on multiple nodes of a cluster. When you SELECT from a `Distributed` table, it rewrites that query, chooses remote nodes according to load balancing settings, and sends the query to them. The `Distributed` table requests remote servers to process a query just up to a stage where intermediate results from different servers can be merged. Then it receives the intermediate results and merges them. The distributed table tries to distribute as much work as possible to remote servers and does not send much intermediate data over the network.
|
||||
|
Loading…
Reference in New Issue
Block a user