Merge pull request #68674 from filimonov/thread_pool_metrics

Thread pool metrics
This commit is contained in:
Sergei Trifonov 2024-09-02 12:19:27 +00:00 committed by GitHub
commit 64af7363c2
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 100 additions and 6 deletions

View File

@ -75,9 +75,9 @@
M(GlobalThread, "Number of threads in global thread pool.") \
M(GlobalThreadActive, "Number of threads in global thread pool running a task.") \
M(GlobalThreadScheduled, "Number of queued or active jobs in global thread pool.") \
M(LocalThread, "Number of threads in local thread pools. The threads in local thread pools are taken from the global thread pool.") \
M(LocalThreadActive, "Number of threads in local thread pools running a task.") \
M(LocalThreadScheduled, "Number of queued or active jobs in local thread pools.") \
M(LocalThread, "Obsolete. Number of threads in local thread pools. The threads in local thread pools are taken from the global thread pool.") \
M(LocalThreadActive, "Obsolete. Number of threads in local thread pools running a task.") \
M(LocalThreadScheduled, "Obsolete. Number of queued or active jobs in local thread pools.") \
M(MergeTreeDataSelectExecutorThreads, "Number of threads in the MergeTreeDataSelectExecutor thread pool.") \
M(MergeTreeDataSelectExecutorThreadsActive, "Number of threads in the MergeTreeDataSelectExecutor thread pool running a task.") \
M(MergeTreeDataSelectExecutorThreadsScheduled, "Number of queued or active jobs in the MergeTreeDataSelectExecutor thread pool.") \

View File

@ -86,6 +86,20 @@
M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
\
M(GlobalThreadPoolExpansions, "Counts the total number of times new threads have been added to the global thread pool. This metric indicates the frequency of expansions in the global thread pool to accommodate increased processing demands.") \
M(GlobalThreadPoolShrinks, "Counts the total number of times the global thread pool has shrunk by removing threads. This occurs when the number of idle threads exceeds max_thread_pool_free_size, indicating adjustments in the global thread pool size in response to decreased thread utilization.") \
M(GlobalThreadPoolThreadCreationMicroseconds, "Total time spent waiting for new threads to start.") \
M(GlobalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the global thread pool.") \
M(GlobalThreadPoolJobs, "Counts the number of jobs that have been pushed to the global thread pool.") \
M(GlobalThreadPoolJobWaitTimeMicroseconds, "Measures the elapsed time from when a job is scheduled in the thread pool to when it is picked up for execution by a worker thread. This metric helps identify delays in job processing, indicating the responsiveness of the thread pool to new tasks.") \
M(LocalThreadPoolExpansions, "Counts the total number of times threads have been borrowed from the global thread pool to expand local thread pools.") \
M(LocalThreadPoolShrinks, "Counts the total number of times threads have been returned to the global thread pool from local thread pools.") \
M(LocalThreadPoolThreadCreationMicroseconds, "Total time local thread pools have spent waiting to borrow a thread from the global pool.") \
M(LocalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the local thread pools.") \
M(LocalThreadPoolJobs, "Counts the number of jobs that have been pushed to the local thread pools.") \
M(LocalThreadPoolBusyMicroseconds, "Total time threads have spent executing the actual work.") \
M(LocalThreadPoolJobWaitTimeMicroseconds, "Measures the elapsed time from when a job is scheduled in the thread pool to when it is picked up for execution by a worker thread. This metric helps identify delays in job processing, indicating the responsiveness of the thread pool to new tasks.") \
\
M(DiskS3GetRequestThrottlerCount, "Number of DiskS3 GET and SELECT requests passed through throttler.") \
M(DiskS3GetRequestThrottlerSleepMicroseconds, "Total time a query was sleeping to conform DiskS3 GET and SELECT request throttling.") \
M(DiskS3PutRequestThrottlerCount, "Number of DiskS3 PUT, COPY, POST and LIST requests passed through throttler.") \

View File

@ -1,4 +1,5 @@
#include <Common/ThreadPool.h>
#include <Common/ProfileEvents.h>
#include <Common/setThreadName.h>
#include <Common/Exception.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
@ -27,6 +28,25 @@ namespace CurrentMetrics
extern const Metric GlobalThreadScheduled;
}
namespace ProfileEvents
{
extern const Event GlobalThreadPoolExpansions;
extern const Event GlobalThreadPoolShrinks;
extern const Event GlobalThreadPoolThreadCreationMicroseconds;
extern const Event GlobalThreadPoolLockWaitMicroseconds;
extern const Event GlobalThreadPoolJobs;
extern const Event GlobalThreadPoolJobWaitTimeMicroseconds;
extern const Event LocalThreadPoolExpansions;
extern const Event LocalThreadPoolShrinks;
extern const Event LocalThreadPoolThreadCreationMicroseconds;
extern const Event LocalThreadPoolLockWaitMicroseconds;
extern const Event LocalThreadPoolJobs;
extern const Event LocalThreadPoolBusyMicroseconds;
extern const Event LocalThreadPoolJobWaitTimeMicroseconds;
}
class JobWithPriority
{
public:
@ -40,6 +60,7 @@ public:
/// Call stacks of all jobs' schedulings leading to this one
std::vector<StackTrace::FramePointers> frame_pointers;
bool enable_job_stack_trace = false;
Stopwatch job_create_time;
JobWithPriority(
Job job_, Priority priority_, CurrentMetrics::Metric metric,
@ -59,6 +80,13 @@ public:
{
return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first
}
UInt64 elapsedMicroseconds() const
{
return job_create_time.elapsedMicroseconds();
}
};
static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool";
@ -180,14 +208,18 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, Priority priority, std:
};
{
Stopwatch watch;
std::unique_lock lock(mutex);
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolLockWaitMicroseconds : ProfileEvents::LocalThreadPoolLockWaitMicroseconds,
watch.elapsedMicroseconds());
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 (wait_microseconds) /// Check for optional. Condition is true if the optional is set. Even if 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));
@ -216,7 +248,13 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, Priority priority, std:
try
{
Stopwatch watch2;
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolThreadCreationMicroseconds : ProfileEvents::LocalThreadPoolThreadCreationMicroseconds,
watch2.elapsedMicroseconds());
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolExpansions : ProfileEvents::LocalThreadPoolExpansions);
}
catch (...)
{
@ -239,6 +277,8 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, Priority priority, std:
/// Wake up a free thread to run the new job.
new_job_or_shutdown.notify_one();
ProfileEvents::increment(std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolJobs : ProfileEvents::LocalThreadPoolJobs);
return static_cast<ReturnType>(true);
}
@ -262,7 +302,14 @@ void ThreadPoolImpl<Thread>::startNewThreadsNoLock()
try
{
Stopwatch watch;
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolThreadCreationMicroseconds : ProfileEvents::LocalThreadPoolThreadCreationMicroseconds,
watch.elapsedMicroseconds());
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolExpansions : ProfileEvents::LocalThreadPoolExpansions);
}
catch (...)
{
@ -293,7 +340,11 @@ void ThreadPoolImpl<Thread>::scheduleOrThrow(Job job, Priority priority, uint64_
template <typename Thread>
void ThreadPoolImpl<Thread>::wait()
{
Stopwatch watch;
std::unique_lock lock(mutex);
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolLockWaitMicroseconds : ProfileEvents::LocalThreadPoolLockWaitMicroseconds,
watch.elapsedMicroseconds());
/// 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.
@ -334,7 +385,11 @@ void ThreadPoolImpl<Thread>::finalize()
/// Wait for all currently running jobs to finish (we don't wait for all scheduled jobs here like the function wait() does).
for (auto & thread : threads)
{
thread.join();
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolShrinks : ProfileEvents::LocalThreadPoolShrinks);
}
threads.clear();
}
@ -391,7 +446,11 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
std::optional<JobWithPriority> job_data;
{
Stopwatch watch;
std::unique_lock lock(mutex);
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolLockWaitMicroseconds : ProfileEvents::LocalThreadPoolLockWaitMicroseconds,
watch.elapsedMicroseconds());
// Finish with previous job if any
if (job_is_done)
@ -424,6 +483,8 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
{
thread_it->detach();
threads.erase(thread_it);
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolShrinks : ProfileEvents::LocalThreadPoolShrinks);
}
return;
}
@ -433,6 +494,10 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
job_data = std::move(const_cast<JobWithPriority &>(jobs.top()));
jobs.pop();
ProfileEvents::increment(
std::is_same_v<Thread, std::thread> ? ProfileEvents::GlobalThreadPoolJobWaitTimeMicroseconds : ProfileEvents::LocalThreadPoolJobWaitTimeMicroseconds,
job_data->elapsedMicroseconds());
/// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them.
if (shutdown)
{
@ -459,7 +524,22 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads);
job_data->job();
if constexpr (!std::is_same_v<Thread, std::thread>)
{
Stopwatch watch;
job_data->job();
// This metric is less relevant for the global thread pool, as it would show large values (time while
// a thread was used by local pools) and increment only when local pools are destroyed.
//
// In cases where global pool threads are used directly (without a local thread pool), distinguishing
// them is difficult.
ProfileEvents::increment(ProfileEvents::LocalThreadPoolBusyMicroseconds, watch.elapsedMicroseconds());
}
else
{
job_data->job();
}
if (thread_trace_context.root_span.isTraceEnabled())
{

View File

@ -131,7 +131,7 @@ private:
bool threads_remove_themselves = true;
const bool shutdown_on_exception = true;
boost::heap::priority_queue<JobWithPriority> jobs;
boost::heap::priority_queue<JobWithPriority,boost::heap::stable<true>> jobs;
std::list<Thread> threads;
std::exception_ptr first_exception;
std::stack<OnDestroyCallback> on_destroy_callbacks;