ClickHouse/src/Common/ThreadPool.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

597 lines
19 KiB
C++
Raw Normal View History

#include <Common/ThreadPool.h>
2022-04-10 23:03:24 +00:00
#include <Common/setThreadName.h>
#include <Common/Exception.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/OpenTelemetryTraceContext.h>
2023-01-30 19:00:48 +00:00
#include <Common/noexcept_scope.h>
#include <type_traits>
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <base/demangle.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_SCHEDULE_TASK;
extern const int LOGICAL_ERROR;
}
}
2019-08-01 20:09:38 +00:00
namespace CurrentMetrics
{
extern const Metric GlobalThread;
extern const Metric GlobalThreadActive;
extern const Metric GlobalThreadScheduled;
2019-08-01 20:09:38 +00:00
}
class JobWithPriority
{
public:
using Job = std::function<void()>;
Job job;
Priority priority;
CurrentMetrics::Increment metric_increment;
DB::OpenTelemetry::TracingContextOnThread thread_trace_context;
/// Call stacks of all jobs' schedulings leading to this one
std::vector<StackTrace::FramePointers> frame_pointers;
bool enable_job_stack_trace = false;
JobWithPriority(
Job job_, Priority priority_, CurrentMetrics::Metric metric,
const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_,
bool capture_frame_pointers)
: job(job_), priority(priority_), metric_increment(metric),
thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers)
{
if (!capture_frame_pointers)
return;
/// Save all previous jobs call stacks and append with current
2024-07-26 08:21:36 +00:00
frame_pointers = DB::Exception::getThreadFramePointers();
frame_pointers.push_back(StackTrace().getFramePointers());
}
bool operator<(const JobWithPriority & rhs) const
{
return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first
}
};
static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool";
template <typename Thread>
ThreadPoolImpl<Thread>::ThreadPoolImpl(Metric metric_threads_, Metric metric_active_threads_, Metric metric_scheduled_jobs_)
: ThreadPoolImpl(metric_threads_, metric_active_threads_, metric_scheduled_jobs_, getNumberOfPhysicalCPUCores())
{
}
template <typename Thread>
ThreadPoolImpl<Thread>::ThreadPoolImpl(
Metric metric_threads_,
Metric metric_active_threads_,
Metric metric_scheduled_jobs_,
size_t max_threads_)
: ThreadPoolImpl(metric_threads_, metric_active_threads_, metric_scheduled_jobs_, max_threads_, max_threads_, max_threads_)
{
}
template <typename Thread>
ThreadPoolImpl<Thread>::ThreadPoolImpl(
Metric metric_threads_,
Metric metric_active_threads_,
Metric metric_scheduled_jobs_,
size_t max_threads_,
size_t max_free_threads_,
size_t queue_size_,
bool shutdown_on_exception_)
: metric_threads(metric_threads_)
, metric_active_threads(metric_active_threads_)
, metric_scheduled_jobs(metric_scheduled_jobs_)
, max_threads(max_threads_)
2023-03-17 09:24:30 +00:00
, max_free_threads(std::min(max_free_threads_, max_threads))
, queue_size(queue_size_ ? std::max(queue_size_, max_threads) : 0 /* zero means the queue is unlimited */)
Do not shutdown global thread pool on exception 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: https://github.com/ClickHouse/ClickHouse/issues/6833#issuecomment-579221732 (Reference to particular comment, since I'm not sure about the initial issue)
2020-02-17 20:15:29 +00:00
, shutdown_on_exception(shutdown_on_exception_)
{
}
template <typename Thread>
void ThreadPoolImpl<Thread>::setMaxThreads(size_t value)
{
std::lock_guard lock(mutex);
bool need_start_threads = (value > max_threads);
bool need_finish_free_threads = (value < max_free_threads);
max_threads = value;
max_free_threads = std::min(max_free_threads, max_threads);
2022-02-04 00:54:33 +00:00
/// We have to also adjust queue size, because it limits the number of scheduled and already running jobs in total.
2023-03-17 09:24:30 +00:00
queue_size = queue_size ? std::max(queue_size, max_threads) : 0;
2022-02-03 23:29:46 +00:00
jobs.reserve(queue_size);
if (need_start_threads)
2023-03-17 09:24:30 +00:00
{
/// Start new threads while there are more scheduled jobs in the queue and the limit `max_threads` is not reached.
startNewThreadsNoLock();
2023-03-17 09:24:30 +00:00
}
else if (need_finish_free_threads)
2023-03-17 09:24:30 +00:00
{
/// Wake up free threads so they can finish themselves.
new_job_or_shutdown.notify_all();
2023-03-17 09:24:30 +00:00
}
}
template <typename Thread>
size_t ThreadPoolImpl<Thread>::getMaxThreads() const
{
std::lock_guard lock(mutex);
return max_threads;
}
template <typename Thread>
void ThreadPoolImpl<Thread>::setMaxFreeThreads(size_t value)
{
std::lock_guard lock(mutex);
bool need_finish_free_threads = (value < max_free_threads);
2023-03-17 09:24:30 +00:00
max_free_threads = std::min(value, max_threads);
if (need_finish_free_threads)
2023-03-17 09:24:30 +00:00
{
/// Wake up free threads so they can finish themselves.
new_job_or_shutdown.notify_all();
2023-03-17 09:24:30 +00:00
}
}
template <typename Thread>
void ThreadPoolImpl<Thread>::setQueueSize(size_t value)
{
std::lock_guard lock(mutex);
2023-03-17 09:24:30 +00:00
queue_size = value ? std::max(value, max_threads) : 0;
/// Reserve memory to get rid of allocations
2021-09-07 12:45:39 +00:00
jobs.reserve(queue_size);
}
template <typename Thread>
template <typename ReturnType>
ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, Priority priority, std::optional<uint64_t> wait_microseconds, bool propagate_opentelemetry_tracing_context)
{
auto on_error = [&](const std::string & reason)
{
if constexpr (std::is_same_v<ReturnType, void>)
{
if (first_exception)
{
std::exception_ptr exception;
std::swap(exception, first_exception);
std::rethrow_exception(exception);
}
throw DB::Exception(DB::ErrorCodes::CANNOT_SCHEDULE_TASK,
"Cannot schedule a task: {} (threads={}, jobs={})", reason,
threads.size(), scheduled_jobs);
}
else
return false;
};
{
std::unique_lock lock(mutex);
2024-04-04 20:32:57 +00:00
if (CannotAllocateThreadFaultInjector::injectFault())
return on_error("fault injected");
auto pred = [this] { return !queue_size || scheduled_jobs < queue_size || shutdown; };
if (wait_microseconds) /// Check for optional. Condition is true if the optional is set and the value is zero.
{
if (!job_finished.wait_for(lock, std::chrono::microseconds(*wait_microseconds), pred))
return on_error(fmt::format("no free thread (timeout={})", *wait_microseconds));
}
else
job_finished.wait(lock, pred);
if (shutdown)
return on_error("shutdown");
2021-05-25 11:54:47 +00:00
/// We must not to allocate any memory after we emplaced a job in a queue.
2021-05-25 12:58:04 +00:00
/// Because if an exception would be thrown, we won't notify a thread about job occurrence.
2021-05-24 16:24:03 +00:00
/// Check if there are enough threads to process job.
if (threads.size() < std::min(max_threads, scheduled_jobs + 1))
{
2021-05-24 16:24:03 +00:00
try
{
threads.emplace_front();
}
catch (...)
{
/// Most likely this is a std::bad_alloc exception
return on_error("cannot allocate thread slot");
2021-05-24 16:24:03 +00:00
}
try
{
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
}
catch (...)
{
threads.pop_front();
return on_error("cannot allocate thread");
}
}
2021-05-24 16:24:03 +00:00
2022-09-01 03:56:10 +00:00
jobs.emplace(std::move(job),
priority,
metric_scheduled_jobs,
2022-09-01 03:56:10 +00:00
/// Tracing context on this thread is used as parent context for the sub-thread that runs the job
propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread(),
/// capture_frame_pointers
DB::Exception::enable_job_stack_trace);
2021-05-24 16:24:03 +00:00
++scheduled_jobs;
}
2021-05-24 16:24:03 +00:00
2023-03-17 20:52:05 +00:00
/// Wake up a free thread to run the new job.
new_job_or_shutdown.notify_one();
return static_cast<ReturnType>(true);
}
template <typename Thread>
void ThreadPoolImpl<Thread>::startNewThreadsNoLock()
{
2023-03-17 20:52:05 +00:00
if (shutdown)
return;
2023-03-17 09:24:30 +00:00
/// Start new threads while there are more scheduled jobs in the queue and the limit `max_threads` is not reached.
while (threads.size() < std::min(scheduled_jobs, max_threads))
{
try
{
threads.emplace_front();
}
catch (...)
{
2023-03-17 09:24:30 +00:00
break; /// failed to start more threads
}
try
{
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
}
catch (...)
{
threads.pop_front();
2023-03-17 09:24:30 +00:00
break; /// failed to start more threads
}
2023-03-17 09:24:30 +00:00
}
}
template <typename Thread>
void ThreadPoolImpl<Thread>::scheduleOrThrowOnError(Job job, Priority priority)
{
scheduleImpl<void>(std::move(job), priority, std::nullopt);
}
template <typename Thread>
bool ThreadPoolImpl<Thread>::trySchedule(Job job, Priority priority, uint64_t wait_microseconds) noexcept
{
return scheduleImpl<bool>(std::move(job), priority, wait_microseconds);
}
template <typename Thread>
void ThreadPoolImpl<Thread>::scheduleOrThrow(Job job, Priority priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context)
{
2022-09-01 03:56:10 +00:00
scheduleImpl<void>(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context);
}
template <typename Thread>
void ThreadPoolImpl<Thread>::wait()
{
std::unique_lock lock(mutex);
/// Signal here just in case.
/// If threads are waiting on condition variables, but there are some jobs in the queue
/// then it will prevent us from deadlock.
new_job_or_shutdown.notify_all();
job_finished.wait(lock, [this] { return scheduled_jobs == 0; });
if (first_exception)
{
std::exception_ptr exception;
std::swap(exception, first_exception);
std::rethrow_exception(exception);
}
}
template <typename Thread>
ThreadPoolImpl<Thread>::~ThreadPoolImpl()
{
2021-08-19 08:22:57 +00:00
/// Note: should not use logger from here,
/// because it can be an instance of GlobalThreadPool that is a global variable
/// and the destruction order of global variables is unspecified.
finalize();
2023-01-30 19:00:48 +00:00
onDestroy();
}
template <typename Thread>
void ThreadPoolImpl<Thread>::finalize()
{
2023-03-17 20:52:05 +00:00
{
std::lock_guard lock(mutex);
shutdown = true;
/// We don't want threads to remove themselves from `threads` anymore, otherwise `thread.join()` will go wrong below in this function.
threads_remove_themselves = false;
}
/// Wake up threads so they can finish themselves.
new_job_or_shutdown.notify_all();
/// Wait for all currently running jobs to finish (we don't wait for all scheduled jobs here like the function wait() does).
2023-03-17 20:52:05 +00:00
for (auto & thread : threads)
thread.join();
threads.clear();
}
2023-01-30 19:00:48 +00:00
template <typename Thread>
void ThreadPoolImpl<Thread>::addOnDestroyCallback(OnDestroyCallback && callback)
{
std::lock_guard lock(mutex);
on_destroy_callbacks.push(std::move(callback));
}
template <typename Thread>
void ThreadPoolImpl<Thread>::onDestroy()
{
while (!on_destroy_callbacks.empty())
{
auto callback = std::move(on_destroy_callbacks.top());
on_destroy_callbacks.pop();
NOEXCEPT_SCOPE({ callback(); });
}
}
template <typename Thread>
size_t ThreadPoolImpl<Thread>::active() const
{
std::lock_guard lock(mutex);
return scheduled_jobs;
}
2021-04-28 18:26:12 +00:00
template <typename Thread>
bool ThreadPoolImpl<Thread>::finished() const
{
std::lock_guard lock(mutex);
2021-04-28 18:26:12 +00:00
return shutdown;
}
template <typename Thread>
void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_it)
{
2021-01-12 14:34:50 +00:00
DENY_ALLOCATIONS_IN_SCOPE;
CurrentMetrics::Increment metric_pool_threads(metric_threads);
2019-08-01 20:09:38 +00:00
bool job_is_done = false;
std::exception_ptr exception_from_job;
2023-03-17 20:52:05 +00:00
/// We'll run jobs in this worker while there are scheduled jobs and until some special event occurs (e.g. shutdown, or decreasing the number of max_threads).
2023-03-17 09:24:30 +00:00
/// And if `max_free_threads > 0` we keep this number of threads even when there are no jobs for them currently.
while (true)
{
2022-04-10 23:03:24 +00:00
/// This is inside the loop to also reset previous thread names set inside the jobs.
setThreadName(DEFAULT_THREAD_NAME);
2022-04-10 23:03:24 +00:00
2023-03-17 20:52:05 +00:00
/// Get a job from the queue.
std::optional<JobWithPriority> job_data;
2023-03-17 20:52:05 +00:00
{
std::unique_lock lock(mutex);
// Finish with previous job if any
if (job_is_done)
{
job_is_done = false;
if (exception_from_job)
{
if (!first_exception)
first_exception = exception_from_job;
if (shutdown_on_exception)
shutdown = true;
exception_from_job = {};
}
--scheduled_jobs;
job_finished.notify_all();
if (shutdown)
new_job_or_shutdown.notify_all(); /// `shutdown` was set, wake up other threads so they can finish themselves.
}
2023-04-13 12:11:06 +00:00
new_job_or_shutdown.wait(lock, [&] { return !jobs.empty() || shutdown || threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads); });
2023-04-13 12:11:06 +00:00
if (jobs.empty() || threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads))
2021-09-06 11:37:51 +00:00
{
2023-04-13 15:17:39 +00:00
// We enter here if:
// - either this thread is not needed anymore due to max_free_threads excess;
// - or shutdown happened AND all jobs are already handled.
2023-04-13 12:11:06 +00:00
if (threads_remove_themselves)
{
thread_it->detach();
threads.erase(thread_it);
}
2021-09-06 11:37:51 +00:00
return;
}
/// boost::priority_queue does not provide interface for getting non-const reference to an element
/// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority.
job_data = std::move(const_cast<JobWithPriority &>(jobs.top()));
jobs.pop();
2023-04-13 12:11:06 +00:00
/// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them.
if (shutdown)
2023-05-05 19:13:50 +00:00
{
{
ALLOW_ALLOCATIONS_IN_SCOPE;
/// job can contain packaged_task which can set exception during destruction
job_data.reset();
}
2023-05-05 19:13:50 +00:00
job_is_done = true;
continue;
2023-05-05 19:13:50 +00:00
}
}
ALLOW_ALLOCATIONS_IN_SCOPE;
2022-09-01 03:56:10 +00:00
/// Set up tracing context for this thread by its parent context.
DB::OpenTelemetry::TracingContextHolder thread_trace_context("ThreadPool::worker()", job_data->thread_trace_context);
/// Run the job.
try
{
if (DB::Exception::enable_job_stack_trace)
2024-07-26 08:21:36 +00:00
DB::Exception::setThreadFramePointers(std::move(job_data->frame_pointers));
CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads);
2019-08-01 20:09:38 +00:00
job_data->job();
if (thread_trace_context.root_span.isTraceEnabled())
{
/// Use the thread name as operation name so that the tracing log will be more clear.
/// The thread name is usually set in jobs, we can only get the name after the job finishes
std::string thread_name = getThreadName();
if (!thread_name.empty() && thread_name != DEFAULT_THREAD_NAME)
{
thread_trace_context.root_span.operation_name = thread_name;
}
else
{
/// If the thread name is not set, use the type name of the job instead
thread_trace_context.root_span.operation_name = demangle(job_data->job.target_type().name());
}
}
/// job should be reset before decrementing scheduled_jobs to
/// ensure that the Job destroyed before wait() returns.
job_data.reset();
2023-03-17 20:52:05 +00:00
}
catch (...)
{
exception_from_job = std::current_exception();
thread_trace_context.root_span.addAttribute(exception_from_job);
/// job should be reset before decrementing scheduled_jobs to
/// ensure that the Job destroyed before wait() returns.
job_data.reset();
}
job_is_done = true;
}
}
template class ThreadPoolImpl<std::thread>;
template class ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>;
template class ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, false>>;
template class ThreadFromGlobalPoolImpl<true, true>;
template class ThreadFromGlobalPoolImpl<true, false>;
template class ThreadFromGlobalPoolImpl<false, false>;
2020-06-22 19:04:12 +00:00
std::unique_ptr<GlobalThreadPool> GlobalThreadPool::the_instance;
GlobalThreadPool::GlobalThreadPool(
size_t max_threads_,
size_t max_free_threads_,
size_t queue_size_,
const bool shutdown_on_exception_,
UInt64 global_profiler_real_time_period_ns_,
UInt64 global_profiler_cpu_time_period_ns_)
: FreeThreadPool(
CurrentMetrics::GlobalThread,
CurrentMetrics::GlobalThreadActive,
CurrentMetrics::GlobalThreadScheduled,
max_threads_,
max_free_threads_,
queue_size_,
shutdown_on_exception_)
, global_profiler_real_time_period_ns(global_profiler_real_time_period_ns_)
, global_profiler_cpu_time_period_ns(global_profiler_cpu_time_period_ns_)
{
}
void GlobalThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size, UInt64 global_profiler_real_time_period_ns, UInt64 global_profiler_cpu_time_period_ns)
{
if (the_instance)
{
2020-09-16 08:59:58 +00:00
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR,
"The global thread pool is initialized twice");
}
the_instance.reset(new GlobalThreadPool(max_threads, max_free_threads, queue_size, false /*shutdown_on_exception*/, global_profiler_real_time_period_ns, global_profiler_cpu_time_period_ns));
2020-06-22 19:04:12 +00:00
}
2020-06-22 19:04:12 +00:00
GlobalThreadPool & GlobalThreadPool::instance()
{
if (!the_instance)
{
2020-06-23 17:31:46 +00:00
// Allow implicit initialization. This is needed for old code that is
// impractical to redo now, especially Arcadia users and unit tests.
initialize();
2020-06-22 19:04:12 +00:00
}
2020-06-22 19:04:12 +00:00
return *the_instance;
}
void GlobalThreadPool::shutdown()
{
if (the_instance)
{
the_instance->finalize();
}
}
2024-04-03 17:49:23 +00:00
CannotAllocateThreadFaultInjector & CannotAllocateThreadFaultInjector::instance()
{
static CannotAllocateThreadFaultInjector ins;
return ins;
}
void CannotAllocateThreadFaultInjector::setFaultProbability(double probability)
{
auto & ins = instance();
std::lock_guard lock(ins.mutex);
ins.enabled = 0 < probability && probability <= 1;
if (ins.enabled)
ins.random.emplace(probability);
else
ins.random.reset();
}
bool CannotAllocateThreadFaultInjector::injectFault()
{
auto & ins = instance();
if (!ins.enabled.load(std::memory_order_relaxed))
return false;
2024-04-03 18:57:12 +00:00
if (ins.block_fault_injections)
return false;
2024-04-03 17:49:23 +00:00
std::lock_guard lock(ins.mutex);
return ins.random && (*ins.random)(ins.rndgen);
}
2024-04-03 18:57:12 +00:00
thread_local bool CannotAllocateThreadFaultInjector::block_fault_injections = false;
scope_guard CannotAllocateThreadFaultInjector::blockFaultInjections()
{
auto & ins = instance();
ins.block_fault_injections = true;
return [&ins](){ ins.block_fault_injections = false; };
}