提交 d854049d 编写于 作者: A Azat Khuzhin

Add tasks/memory metrics for distributed/buffer schedule pools

Follow-up-for: #10315
Follow-up-for: #10263
上级 c04c73b4
......@@ -12,6 +12,8 @@
M(BackgroundPoolTask, "Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping)") \
M(BackgroundMovePoolTask, "Number of active tasks in BackgroundProcessingPool for moves") \
M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc.") \
M(BackgroundBufferFlushSchedulePoolTask, "Number of active tasks in BackgroundBufferFlushSchedulePool. This pool is used for periodic Buffer flushes") \
M(BackgroundDistributedSchedulePoolTask, "Number of active tasks in BackgroundDistributedSchedulePool. This pool is used for distributed sends that is done in background.") \
M(CacheDictionaryUpdateQueueBatches, "Number of 'batches' (a set of keys) in update queue in CacheDictionaries.") \
M(CacheDictionaryUpdateQueueKeys, "Exact number of keys in update queue in CacheDictionaries.") \
M(DiskSpaceReservedForMerge, "Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts.") \
......@@ -34,6 +36,8 @@
M(MemoryTrackingInBackgroundProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for backround merges, mutations and fetches). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \
M(MemoryTrackingInBackgroundMoveProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for backround moves). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \
M(MemoryTrackingInBackgroundSchedulePool, "Total amount of memory (bytes) allocated in background schedule pool (that is dedicated for bookkeeping tasks of Replicated tables).") \
M(MemoryTrackingInBackgroundBufferFlushSchedulePool, "Total amount of memory (bytes) allocated in background buffer flushes pool (that is dedicated for background buffer flushes).") \
M(MemoryTrackingInBackgroundDistributedSchedulePool, "Total amount of memory (bytes) allocated in background distributed schedule pool (that is dedicated for distributed sends).") \
M(MemoryTrackingForMerges, "Total amount of memory (bytes) allocated for background merges. Included in MemoryTrackingInBackgroundProcessingPool. Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \
M(LeaderElection, "Number of Replicas participating in leader election. Equals to total number of replicas in usual cases.") \
M(EphemeralNode, "Number of ephemeral nodes hold in ZooKeeper.") \
......
#include "BackgroundSchedulePool.h"
#include <Common/MemoryTracker.h>
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
......@@ -10,12 +9,6 @@
#include <ext/scope_guard.h>
namespace CurrentMetrics
{
extern const Metric BackgroundSchedulePoolTask;
extern const Metric MemoryTrackingInBackgroundSchedulePool;
}
namespace DB
{
......@@ -95,7 +88,7 @@ bool BackgroundSchedulePoolTaskInfo::activateAndSchedule()
void BackgroundSchedulePoolTaskInfo::execute()
{
Stopwatch watch;
CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundSchedulePoolTask};
CurrentMetrics::Increment metric_increment{pool.tasks_metric};
std::lock_guard lock_exec(exec_mutex);
......@@ -155,8 +148,10 @@ Coordination::WatchCallback BackgroundSchedulePoolTaskInfo::getWatchCallback()
}
BackgroundSchedulePool::BackgroundSchedulePool(size_t size_)
BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Metric tasks_metric_, CurrentMetrics::Metric memory_metric_)
: size(size_)
, tasks_metric(tasks_metric_)
, memory_metric(memory_metric_)
{
LOG_INFO(&Logger::get("BackgroundSchedulePool"), "Create BackgroundSchedulePool with " << size << " threads");
......@@ -252,7 +247,7 @@ void BackgroundSchedulePool::threadFunction()
attachToThreadGroup();
SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); });
if (auto * memory_tracker = CurrentThread::getMemoryTracker())
memory_tracker->setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool);
memory_tracker->setMetric(memory_metric);
while (!shutdown)
{
......
......@@ -12,6 +12,7 @@
#include <functional>
#include <boost/noncopyable.hpp>
#include <Common/ZooKeeper/Types.h>
#include <Common/CurrentMetrics.h>
#include <Common/CurrentThread.h>
#include <Common/ThreadPool.h>
......@@ -49,7 +50,7 @@ public:
size_t getNumberOfThreads() const { return size; }
BackgroundSchedulePool(size_t size_);
BackgroundSchedulePool(size_t size_, CurrentMetrics::Metric tasks_metric_, CurrentMetrics::Metric memory_metric_);
~BackgroundSchedulePool();
private:
......@@ -82,6 +83,9 @@ private:
/// Thread group used for profiling purposes
ThreadGroupStatusPtr thread_group;
CurrentMetrics::Metric tasks_metric;
CurrentMetrics::Metric memory_metric;
void attachToThreadGroup();
};
......
......@@ -70,6 +70,15 @@ namespace CurrentMetrics
extern const Metric ContextLockWait;
extern const Metric BackgroundMovePoolTask;
extern const Metric MemoryTrackingInBackgroundMoveProcessingPool;
extern const Metric BackgroundSchedulePoolTask;
extern const Metric MemoryTrackingInBackgroundSchedulePool;
extern const Metric BackgroundBufferFlushSchedulePoolTask;
extern const Metric MemoryTrackingInBackgroundBufferFlushSchedulePool;
extern const Metric BackgroundDistributedSchedulePoolTask;
extern const Metric MemoryTrackingInBackgroundDistributedSchedulePool;
}
......@@ -1351,7 +1360,10 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool()
{
auto lock = getLock();
if (!shared->buffer_flush_schedule_pool)
shared->buffer_flush_schedule_pool.emplace(settings.background_buffer_flush_schedule_pool_size);
shared->buffer_flush_schedule_pool.emplace(
settings.background_buffer_flush_schedule_pool_size,
CurrentMetrics::BackgroundBufferFlushSchedulePoolTask,
CurrentMetrics::MemoryTrackingInBackgroundBufferFlushSchedulePool);
return *shared->buffer_flush_schedule_pool;
}
......@@ -1359,7 +1371,10 @@ BackgroundSchedulePool & Context::getSchedulePool()
{
auto lock = getLock();
if (!shared->schedule_pool)
shared->schedule_pool.emplace(settings.background_schedule_pool_size);
shared->schedule_pool.emplace(
settings.background_schedule_pool_size,
CurrentMetrics::BackgroundSchedulePoolTask,
CurrentMetrics::MemoryTrackingInBackgroundSchedulePool);
return *shared->schedule_pool;
}
......@@ -1367,7 +1382,10 @@ BackgroundSchedulePool & Context::getDistributedSchedulePool()
{
auto lock = getLock();
if (!shared->distributed_schedule_pool)
shared->distributed_schedule_pool.emplace(settings.background_distributed_schedule_pool_size);
shared->distributed_schedule_pool.emplace(
settings.background_distributed_schedule_pool_size,
CurrentMetrics::BackgroundDistributedSchedulePoolTask,
CurrentMetrics::MemoryTrackingInBackgroundDistributedSchedulePool);
return *shared->distributed_schedule_pool;
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册