From 25ef095e21388671e06fdd6c5e3cb84428db5566 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 8 Sep 2023 13:52:15 +0800 Subject: [PATCH 01/20] refine MinTSOScheduler for resource control Signed-off-by: guo-shaoge --- dbms/src/Common/TiFlashMetrics.cpp | 14 ++ dbms/src/Common/TiFlashMetrics.h | 17 +- dbms/src/Flash/Mpp/MPPTaskManager.cpp | 4 +- dbms/src/Flash/Mpp/MPPTaskManager.h | 2 +- dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp | 2 +- dbms/src/Flash/Mpp/MPPTaskScheduleEntry.h | 2 + dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 178 +++++++++++------- dbms/src/Flash/Mpp/MinTSOScheduler.h | 47 +++-- dbms/src/Flash/tests/gtest_compute_server.cpp | 132 +++---------- 9 files changed, 189 insertions(+), 209 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.cpp b/dbms/src/Common/TiFlashMetrics.cpp index 93edec8d10c..0d21ade984a 100644 --- a/dbms/src/Common/TiFlashMetrics.cpp +++ b/dbms/src/Common/TiFlashMetrics.cpp @@ -58,6 +58,20 @@ TiFlashMetrics::TiFlashMetrics() .Name("tiflash_storage_sync_replica_ru") .Help("RU for synchronous replica of keyspace") .Register(*registry); + registered_min_tso_scheduler_family + = &prometheus::BuildGauge().Name("tiflash_task_scheduler").Help("Min-tso task scheduler").Register(*registry); +} + +prometheus::Gauge * TiFlashMetrics::getOrCreateMinTSOGauge(const String & resource_group_name, const String & type) +{ + const String key = resource_group_name + type; + std::map labels{{"resource_group", resource_group_name}, {"type", type}}; + auto iter = registered_resource_group_min_tso.find(key); + if (iter != registered_resource_group_min_tso.end()) + { + return iter->second; + } + return registered_resource_group_min_tso[key] = ®istered_min_tso_scheduler_family->Add(labels); } void TiFlashMetrics::addReplicaSyncRU(UInt32 keyspace_id, UInt64 ru) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index a635bf83c2f..8a0a6e95399 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -468,18 +468,6 @@ namespace DB F(type_active_threads_of_dispatch_mpp, {"type", "rpc_dispatch_mpp"}), \ F(type_active_rpc_async_worker, {"type", "rpc_async_worker_active"}), \ F(type_total_rpc_async_worker, {"type", "rpc_async_worker_total"})) \ - M(tiflash_task_scheduler, \ - "Min-tso task scheduler", \ - Gauge, \ - F(type_min_tso, {"type", "min_tso"}), \ - F(type_waiting_queries_count, {"type", "waiting_queries_count"}), \ - F(type_active_queries_count, {"type", "active_queries_count"}), \ - F(type_waiting_tasks_count, {"type", "waiting_tasks_count"}), \ - F(type_active_tasks_count, {"type", "active_tasks_count"}), \ - F(type_estimated_thread_usage, {"type", "estimated_thread_usage"}), \ - F(type_thread_soft_limit, {"type", "thread_soft_limit"}), \ - F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ - F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) \ M(tiflash_task_scheduler_waiting_duration_seconds, \ "Bucketed histogram of task waiting for scheduling duration", \ Histogram, \ @@ -822,6 +810,8 @@ class TiFlashMetrics void addReplicaSyncRU(UInt32 keyspace_id, UInt64 ru); + prometheus::Gauge * getOrCreateMinTSOGauge(const String & resource_group_name, const String & type); + private: TiFlashMetrics(); @@ -851,6 +841,9 @@ class TiFlashMetrics std::mutex replica_sync_ru_mtx; std::unordered_map registered_keyspace_sync_replica_ru; + prometheus::Family * registered_min_tso_scheduler_family; + std::unordered_map registered_resource_group_min_tso; + public: #define MAKE_METRIC_MEMBER_M(family_name, help, type, ...) \ MetricFamily family_name \ diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.cpp b/dbms/src/Flash/Mpp/MPPTaskManager.cpp index 88821c622be..c8b9dba7b78 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskManager.cpp @@ -471,9 +471,9 @@ bool MPPTaskManager::tryToScheduleTask(MPPTaskScheduleEntry & schedule_entry) return scheduler->tryToSchedule(schedule_entry, *this); } -void MPPTaskManager::releaseThreadsFromScheduler(const int needed_threads) +void MPPTaskManager::releaseThreadsFromScheduler(const String & resource_group_name, const int needed_threads) { std::lock_guard lock(mu); - scheduler->releaseThreadsThenSchedule(needed_threads, *this); + scheduler->releaseThreadsThenSchedule(resource_group_name, needed_threads, *this); } } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.h b/dbms/src/Flash/Mpp/MPPTaskManager.h index 94ca98bac99..29aa091f0f2 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.h +++ b/dbms/src/Flash/Mpp/MPPTaskManager.h @@ -226,7 +226,7 @@ class MPPTaskManager : private boost::noncopyable bool tryToScheduleTask(MPPTaskScheduleEntry & schedule_entry); - void releaseThreadsFromScheduler(int needed_threads); + void releaseThreadsFromScheduler(const String & resource_group_name, int needed_threads); std::pair findTunnelWithTimeout( const ::mpp::EstablishMPPConnectionRequest * request, diff --git a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp index e872e05dd24..e46135c79af 100644 --- a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp @@ -31,7 +31,7 @@ MPPTaskScheduleEntry::~MPPTaskScheduleEntry() { if (schedule_state == ScheduleState::SCHEDULED) { - manager->releaseThreadsFromScheduler(needed_threads); + manager->releaseThreadsFromScheduler(getResourceGroupName(), needed_threads); schedule_state = ScheduleState::COMPLETED; } } diff --git a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.h b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.h index dcf8033f5ea..baecc2e4e9d 100644 --- a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.h +++ b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.h @@ -41,6 +41,8 @@ class MPPTaskScheduleEntry const MPPTaskId & getMPPTaskId() const; + const String & getResourceGroupName() const { return id.gather_id.query_id.resource_group_name; } + ~MPPTaskScheduleEntry(); MPPTaskScheduleEntry(MPPTaskManager * manager_, const MPPTaskId & id_); diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 0e8e074611f..2744a264e02 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -27,12 +27,14 @@ extern const char random_min_tso_scheduler_failpoint[]; constexpr UInt64 OS_THREAD_SOFT_LIMIT = 100000; -MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit_) - : min_query_id(MPPTaskId::Max_Query_Id) - , thread_soft_limit(soft_limit) +#define GET_MIN_TSO_METRIC(resource_group_name, type) \ + TiFlashMetrics::instance().getOrCreateMinTSOGauge(resource_group_name, type) + +MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit) + : thread_soft_limit(soft_limit) , thread_hard_limit(hard_limit) - , estimated_thread_usage(0) - , active_set_soft_limit(active_set_soft_limit_) + , global_estimated_thread_usage(0) + , active_set_soft_limit(active_set_soft_limit) , log(Logger::get()) { auto cores = static_cast(getNumberOfLogicalCPUCores() / 2); @@ -73,15 +75,18 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac thread_soft_limit, active_set_soft_limit); } - GET_METRIC(tiflash_task_scheduler, type_min_tso).Set(min_query_id.query_ts); - GET_METRIC(tiflash_task_scheduler, type_thread_soft_limit).Set(thread_soft_limit); - GET_METRIC(tiflash_task_scheduler, type_thread_hard_limit).Set(thread_hard_limit); - GET_METRIC(tiflash_task_scheduler, type_estimated_thread_usage).Set(estimated_thread_usage); - GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(0); - GET_METRIC(tiflash_task_scheduler, type_active_queries_count).Set(0); - GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Set(0); - GET_METRIC(tiflash_task_scheduler, type_active_tasks_count).Set(0); - GET_METRIC(tiflash_task_scheduler, type_hard_limit_exceeded_count).Set(0); + + const auto & empty_detail = getOrCreateSchedulerDetail(""); + GET_MIN_TSO_METRIC("", "min_tso")->Set(empty_detail.min_query_id.query_ts); + GET_MIN_TSO_METRIC("", "thread_soft_limit")->Set(thread_soft_limit); + GET_MIN_TSO_METRIC("", "thread_hard_limit")->Set(thread_hard_limit); + GET_MIN_TSO_METRIC("", "estimated_thread_usage")->Set(empty_detail.estimated_thread_usage); + GET_MIN_TSO_METRIC("", "global_estimated_thread_usage")->Set(global_estimated_thread_usage); + GET_MIN_TSO_METRIC("", "waiting_queries_count")->Set(0); + GET_MIN_TSO_METRIC("", "active_queries_count")->Set(0); + GET_MIN_TSO_METRIC("", "waiting_tasks_count")->Set(0); + GET_MIN_TSO_METRIC("", "active_tasks_count")->Set(0); + GET_MIN_TSO_METRIC("", "hard_limit_exceeded_count")->Set(0); } } @@ -108,7 +113,7 @@ bool MinTSOScheduler::tryToSchedule(MPPTaskScheduleEntry & schedule_entry, MPPTa void MinTSOScheduler::deleteQuery( const MPPQueryId & query_id, MPPTaskManager & task_manager, - const bool is_cancelled, + bool is_cancelled, Int64 gather_id) { if (isDisabled()) @@ -116,6 +121,7 @@ void MinTSOScheduler::deleteQuery( return; } + auto & detail = mustGetSchedulerDetail(query_id.resource_group_name); bool all_gathers_deleted = true; auto query = task_manager.getMPPQueryWithoutLock(query_id); @@ -133,7 +139,7 @@ void MinTSOScheduler::deleteQuery( if (task != nullptr) task->scheduleThisTask(ScheduleState::FAILED); gather_it.second->waiting_tasks.pop(); - GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Decrement(); + GET_MIN_TSO_METRIC(query_id.resource_group_name, "waiting_tasks_count")->Decrement(); } } } @@ -151,19 +157,20 @@ void MinTSOScheduler::deleteQuery( "{} query {} (is min = {}) is deleted from active set {} left {} or waiting set {} left {}.", is_cancelled ? "Cancelled" : "Finished", query_id.toString(), - query_id == min_query_id, - active_set.find(query_id) != active_set.end(), - active_set.size(), - waiting_set.find(query_id) != waiting_set.end(), - waiting_set.size()); - active_set.erase(query_id); - waiting_set.erase(query_id); - GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); - GET_METRIC(tiflash_task_scheduler, type_active_queries_count).Set(active_set.size()); + query_id == detail.min_query_id, + detail.active_set.find(query_id) != detail.active_set.end(), + detail.active_set.size(), + detail.waiting_set.find(query_id) != detail.waiting_set.end(), + detail.waiting_set.size()); + detail.active_set.erase(query_id); + detail.waiting_set.erase(query_id); + GET_MIN_TSO_METRIC(query_id.resource_group_name, "waiting_queries_count")->Set(detail.waiting_set.size()); + GET_MIN_TSO_METRIC(query_id.resource_group_name, "active_queries_count")->Set(detail.active_set.size()); + /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. - if (updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it")) + if (detail.updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it", log)) { - scheduleWaitingQueries(task_manager); + scheduleWaitingQueries(detail, task_manager, log); } } else @@ -178,42 +185,51 @@ void MinTSOScheduler::deleteQuery( } /// NOTE: should not throw exceptions due to being called when destruction. -void MinTSOScheduler::releaseThreadsThenSchedule(const int needed_threads, MPPTaskManager & task_manager) +void MinTSOScheduler::releaseThreadsThenSchedule( + const String & resource_group_name, + int needed_threads, + MPPTaskManager & task_manager) { if (isDisabled()) { return; } - auto updated_estimated_threads = static_cast(estimated_thread_usage) - needed_threads; + auto & detail = mustGetSchedulerDetail(resource_group_name); + auto updated_estimated_threads = static_cast(detail.estimated_thread_usage) - needed_threads; RUNTIME_ASSERT( updated_estimated_threads >= 0, log, "estimated_thread_usage should not be smaller than 0, actually is {}.", updated_estimated_threads); - estimated_thread_usage = updated_estimated_threads; - GET_METRIC(tiflash_task_scheduler, type_estimated_thread_usage).Set(estimated_thread_usage); - GET_METRIC(tiflash_task_scheduler, type_active_tasks_count).Decrement(); + detail.estimated_thread_usage = updated_estimated_threads; + GET_MIN_TSO_METRIC(resource_group_name, "estimated_thread_usage")->Set(detail.estimated_thread_usage); + GET_MIN_TSO_METRIC(resource_group_name, "active_tasks_count")->Decrement(); /// as tasks release some threads, so some tasks would get scheduled. - scheduleWaitingQueries(task_manager); + scheduleWaitingQueries(detail, task_manager, log); + if (detail.active_set.size() + detail.waiting_set.size() == 0) + { + LOG_INFO(log, "min tso scheduler_detail of resouce group {} deleted", resource_group_name); + scheduler_details.erase(resource_group_name); + } } -void MinTSOScheduler::scheduleWaitingQueries(MPPTaskManager & task_manager) +void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskManager & task_manager, LoggerPtr log) { /// schedule new tasks - while (!waiting_set.empty()) + while (!detail.waiting_set.empty()) { - auto current_query_id = *waiting_set.begin(); + auto current_query_id = *detail.waiting_set.begin(); auto query = task_manager.getMPPQueryWithoutLock(current_query_id); if (nullptr == query) /// silently solve this rare case { LOG_ERROR(log, "the waiting query {} is not in the task manager.", current_query_id.toString()); - updateMinQueryId(current_query_id, true, "as it is not in the task manager."); - active_set.erase(current_query_id); - waiting_set.erase(current_query_id); - GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); - GET_METRIC(tiflash_task_scheduler, type_active_queries_count).Set(active_set.size()); + detail.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); + detail.active_set.erase(current_query_id); + detail.waiting_set.erase(current_query_id); + GET_MIN_TSO_METRIC(detail.resource_group_name, "waiting_queries_count")->Set(detail.waiting_set.size()); + GET_MIN_TSO_METRIC(detail.resource_group_name, "active_queries_count")->Set(detail.active_set.size()); continue; } @@ -221,8 +237,8 @@ void MinTSOScheduler::scheduleWaitingQueries(MPPTaskManager & task_manager) log, "query {} (is min = {}) is to be scheduled from waiting set (size = {}).", current_query_id.toString(), - current_query_id == min_query_id, - waiting_set.size()); + current_query_id == detail.min_query_id, + detail.waiting_set.size()); /// schedule tasks one by one for (auto & gather_set : query->mpp_gathers) { @@ -268,42 +284,46 @@ bool MinTSOScheduler::scheduleImp( const MPPQueryId & query_id, const MPPGatherTaskSetPtr & query_task_set, MPPTaskScheduleEntry & schedule_entry, - const bool isWaiting, + bool isWaiting, bool & has_error) { auto needed_threads = schedule_entry.getNeededThreads(); + auto & detail = getOrCreateSchedulerDetail(query_id.resource_group_name); + auto check_for_new_min_tso - = query_id <= min_query_id && estimated_thread_usage + needed_threads <= thread_hard_limit; - auto check_for_not_min_tso - = (active_set.size() < active_set_soft_limit || active_set.find(query_id) != active_set.end()) - && (estimated_thread_usage + needed_threads <= thread_soft_limit); + = query_id <= detail.min_query_id && global_estimated_thread_usage + needed_threads <= thread_hard_limit; + auto check_for_not_min_tso = (detail.active_set.size() < active_set_soft_limit + || detail.active_set.find(query_id) != detail.active_set.end()) + && (detail.estimated_thread_usage + needed_threads <= thread_soft_limit); if (check_for_new_min_tso || check_for_not_min_tso) { - updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it"); - active_set.insert(query_id); + detail.updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it", log); + detail.active_set.insert(query_id); if (schedule_entry.schedule(ScheduleState::SCHEDULED)) { - estimated_thread_usage += needed_threads; - GET_METRIC(tiflash_task_scheduler, type_active_tasks_count).Increment(); + detail.estimated_thread_usage += needed_threads; + global_estimated_thread_usage += needed_threads; + GET_MIN_TSO_METRIC(detail.resource_group_name, "active_tasks_count")->Increment(); } - GET_METRIC(tiflash_task_scheduler, type_active_queries_count).Set(active_set.size()); - GET_METRIC(tiflash_task_scheduler, type_estimated_thread_usage).Set(estimated_thread_usage); + GET_MIN_TSO_METRIC(detail.resource_group_name, "active_queries_count")->Set(detail.active_set.size()); + GET_MIN_TSO_METRIC(detail.resource_group_name, "estimated_thread_usage")->Set(detail.estimated_thread_usage); + GET_MIN_TSO_METRIC("", "global_estimated_thread_usage")->Set(global_estimated_thread_usage); LOG_DEBUG( log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} " "of the thread {} limit {}.", schedule_entry.getMPPTaskId().toString(), - active_set.size(), + detail.active_set.size(), isWaiting ? "from the waiting set" : "directly", needed_threads, - estimated_thread_usage, - min_query_id == query_id ? "hard" : "soft", - min_query_id == query_id ? thread_hard_limit : thread_soft_limit); + detail.estimated_thread_usage, + detail.min_query_id == query_id ? "hard" : "soft", + detail.min_query_id == query_id ? thread_hard_limit : thread_soft_limit); return true; } else { - bool is_query_id_min = query_id <= min_query_id; + bool is_query_id_min = query_id <= detail.min_query_id; fiu_do_on(FailPoints::random_min_tso_scheduler_failpoint, is_query_id_min = true;); if (is_query_id_min) /// the min_query_id query should fully run, otherwise throw errors here. { @@ -335,10 +355,10 @@ bool MinTSOScheduler::scheduleImp( } if (!isWaiting) { - waiting_set.insert(query_id); + detail.waiting_set.insert(query_id); query_task_set->waiting_tasks.push(schedule_entry.getMPPTaskId()); - GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); - GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Increment(); + GET_MIN_TSO_METRIC(detail.resource_group_name, "waiting_queries_count")->Set(detail.waiting_set.size()); + GET_MIN_TSO_METRIC(detail.resource_group_name, "waiting_tasks_count")->Increment(); } LOG_INFO( log, @@ -347,16 +367,20 @@ bool MinTSOScheduler::scheduleImp( "required threads count are {}, waiting set size = {}", query_id.toString(), !isWaiting, - thread_soft_limit - estimated_thread_usage, - active_set_soft_limit - active_set.size(), + thread_soft_limit - detail.estimated_thread_usage, + active_set_soft_limit - detail.active_set.size(), needed_threads, - waiting_set.size()); + detail.waiting_set.size()); return false; } } /// if return true, then need to schedule the waiting tasks of the min_query_id. -bool MinTSOScheduler::updateMinQueryId(const MPPQueryId & query_id, const bool retired, const String & msg) +bool MinTSOScheduler::SchedulerDetail::updateMinQueryId( + const MPPQueryId & query_id, + bool retired, + const String & msg, + LoggerPtr log) { auto old_min_query_id = min_query_id; bool force_scheduling = false; @@ -378,8 +402,8 @@ bool MinTSOScheduler::updateMinQueryId(const MPPQueryId & query_id, const bool r if (min_query_id != old_min_query_id) /// if min_query_id == MPPTaskId::Max_Query_Id and the query_id is not to be cancelled, the used_threads, active_set.size() and waiting_set.size() must be 0. { - GET_METRIC(tiflash_task_scheduler, type_min_tso) - .Set(min_query_id.query_ts == 0 ? min_query_id.start_ts : min_query_id.query_ts); + GET_MIN_TSO_METRIC(resource_group_name, "min_tso") + ->Set(min_query_id.query_ts == 0 ? min_query_id.start_ts : min_query_id.query_ts); LOG_DEBUG( log, "min_query_id query is updated from {} to {} {}, used threads = {}, {} active and {} waiting queries.", @@ -393,4 +417,22 @@ bool MinTSOScheduler::updateMinQueryId(const MPPQueryId & query_id, const bool r return force_scheduling; } +MinTSOScheduler::SchedulerDetail & MinTSOScheduler::mustGetSchedulerDetail(const String & resource_group_name) +{ + auto iter = scheduler_details.find(resource_group_name); + RUNTIME_CHECK(iter != scheduler_details.end()); + return iter->second; +} + +MinTSOScheduler::SchedulerDetail & MinTSOScheduler::getOrCreateSchedulerDetail(const String & resource_group_name) +{ + const String name_with_prefix = "rg_" + resource_group_name; + auto iter = scheduler_details.find(resource_group_name); + if (iter == scheduler_details.end()) + { + iter = scheduler_details.insert({resource_group_name, SchedulerDetail(resource_group_name)}).first; + } + return iter->second; +} + } // namespace DB diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.h b/dbms/src/Flash/Mpp/MinTSOScheduler.h index 48da9315236..cf44f6cbe4e 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.h +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.h @@ -35,7 +35,7 @@ using MPPGatherTaskSetPtr = std::shared_ptr; class MinTSOScheduler : private boost::noncopyable { public: - MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit_); + MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit); ~MinTSOScheduler() = default; /// try to schedule this task if it is the min_query_id query or there are enough threads, otherwise put it into the waiting set. /// NOTE: call tryToSchedule under the lock protection of MPPTaskManager @@ -43,31 +43,48 @@ class MinTSOScheduler : private boost::noncopyable /// delete this to-be cancelled/finished query from scheduler and update min_query_id if needed, so that there aren't cancelled/finished queries in the scheduler. /// NOTE: call deleteQuery under the lock protection of MPPTaskManager - void deleteQuery( - const MPPQueryId & query_id, - MPPTaskManager & task_manager, - const bool is_cancelled, - Int64 gather_id); + void deleteQuery(const MPPQueryId & query_id, MPPTaskManager & task_manager, bool is_cancelled, Int64 gather_id); /// all scheduled tasks should finally call this function to release threads and schedule new tasks - void releaseThreadsThenSchedule(const int needed_threads, MPPTaskManager & task_manager); + void releaseThreadsThenSchedule( + const String & resource_group_name, + int needed_threads, + MPPTaskManager & task_manager); private: bool scheduleImp( const MPPQueryId & query_id, const MPPGatherTaskSetPtr & query_task_set, MPPTaskScheduleEntry & schedule_entry, - const bool isWaiting, + bool isWaiting, bool & has_error); - bool updateMinQueryId(const MPPQueryId & query_id, const bool retired, const String & msg); - void scheduleWaitingQueries(MPPTaskManager & task_manager); - bool isDisabled() { return thread_hard_limit == 0 && thread_soft_limit == 0; } - std::set waiting_set; - std::set active_set; - MPPQueryId min_query_id; + bool isDisabled() const { return thread_hard_limit == 0 && thread_soft_limit == 0; } + + struct SchedulerDetail + { + explicit SchedulerDetail(const String & name) + : resource_group_name(name) + , min_query_id(MPPTaskId::Max_Query_Id) + , estimated_thread_usage(0) + {} + + const String resource_group_name; + std::set waiting_set; + std::set active_set; + MPPQueryId min_query_id; + UInt64 estimated_thread_usage; + + bool updateMinQueryId(const MPPQueryId & query_id, bool retired, const String & msg, LoggerPtr log); + }; + + void scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskManager & task_manager, LoggerPtr log); + SchedulerDetail & mustGetSchedulerDetail(const String & resource_group_name); + SchedulerDetail & getOrCreateSchedulerDetail(const String & resource_group_name); + + std::unordered_map scheduler_details; UInt64 thread_soft_limit; UInt64 thread_hard_limit; - UInt64 estimated_thread_usage; + UInt64 global_estimated_thread_usage; /// to prevent from too many queries just issue a part of tasks to occupy threads, in proportion to the hardware cores. size_t active_set_soft_limit; LoggerPtr log; diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index 9f7ac98562b..ca60814a064 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -945,16 +945,8 @@ try context.context->setSetting("task_scheduler_active_set_soft_limit", active_set_soft_limit); startServers(1); setCancelTest(); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 0); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); std::vector running_queries; std::vector gather_ids; try @@ -966,28 +958,12 @@ try } using namespace std::literals::chrono_literals; std::this_thread::sleep_for(2s); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 2); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); addOneQuery(1, running_queries, gather_ids); std::this_thread::sleep_for(2s); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 3); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 3); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); for (const auto & gather_id : gather_ids) MockComputeServerManager::instance().cancelGather(gather_id); for (auto & t : running_queries) @@ -1001,42 +977,18 @@ try } using namespace std::literals::chrono_literals; std::this_thread::sleep_for(2s); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 2); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); addOneQuery(30, running_queries, gather_ids); std::this_thread::sleep_for(2s); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 2); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 1); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 1); /// cancel 1 running query MockComputeServerManager::instance().cancelGather(gather_ids[0]); running_queries[0].join(); std::this_thread::sleep_for(2s); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 2); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); for (size_t i = 1; i < running_queries.size(); i++) MockComputeServerManager::instance().cancelGather(gather_ids[i]); for (size_t i = 1; i < running_queries.size(); i++) @@ -1059,16 +1011,8 @@ try { startServers(1); setCancelTest(); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 0); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); std::vector running_queries; std::vector gather_ids; auto multiple_gathers_properties = DB::tests::getDAGPropertiesForTest(serverNum(), 1, 1, 1); @@ -1085,16 +1029,8 @@ try using namespace std::literals::chrono_literals; std::this_thread::sleep_for(2s); /// 6 gathers, but two query - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 2); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); std::vector killed_gathers{0, 2, 4}; std::vector remaining_gathers{1, 3}; for (const auto i : killed_gathers) @@ -1108,46 +1044,22 @@ try assertGatherActive(gather_ids[i]); } /// the active query count should not change - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 2); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); /// kill single gather query MockComputeServerManager::instance().cancelGather(gather_ids[5]); assertGatherCancelled(gather_ids[5]); /// the active query count should be 1 - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 1); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 1); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); /// kill the rest gathers for (const auto i : remaining_gathers) { MockComputeServerManager::instance().cancelGather(gather_ids[i]); assertGatherCancelled(gather_ids[i]); } - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count) - .Value() - == 0); - ASSERT_TRUE( - TiFlashMetrics::instance() - .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count) - .Value() - == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 0); + ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); for (auto & t : running_queries) if (t.joinable()) t.join(); From d7335e24b3c88581ae4c24914db80368a3a70b5d Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 10:46:58 +0800 Subject: [PATCH 02/20] refine metric Signed-off-by: guo-shaoge --- dbms/src/Common/TiFlashMetrics.h | 12 ++- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 104 +++++++++++++++---------- 2 files changed, 75 insertions(+), 41 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 1b94b494fd8..0787cf9bea3 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -694,7 +694,17 @@ namespace DB F(type_total_consumption, {"type", "total_consumption"}), \ F(type_bucket_fill_rate, {"type", "bucket_fill_rate"}), \ F(type_bucket_capacity, {"type", "bucket_capacity"}), \ - F(type_fetch_tokens_from_gac_count, {"type", "fetch_tokens_from_gac_count"})) + F(type_fetch_tokens_from_gac_count, {"type", "fetch_tokens_from_gac_count"}), \ + F(type_min_tso, {"type", "min_tso"}), \ + F(type_waiting_queries_count, {"type", "waiting_queries_count"}), \ + F(type_active_queries_count, {"type", "active_queries_count"}), \ + F(type_waiting_tasks_count, {"type", "waiting_tasks_count"}), \ + F(type_active_tasks_count, {"type", "active_tasks_count"}), \ + F(type_estimated_thread_usage, {"type", "estimated_thread_usage"}), \ + F(type_global_estimated_thread_usage, {"type", "global_estimated_thread_usage"}), \ + F(type_thread_soft_limit, {"type", "thread_soft_limit"}), \ + F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ + F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) /// Buckets with boundaries [start * base^0, start * base^1, ..., start * base^(size-1)] diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 2744a264e02..36cbc33d42a 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -27,9 +27,6 @@ extern const char random_min_tso_scheduler_failpoint[]; constexpr UInt64 OS_THREAD_SOFT_LIMIT = 100000; -#define GET_MIN_TSO_METRIC(resource_group_name, type) \ - TiFlashMetrics::instance().getOrCreateMinTSOGauge(resource_group_name, type) - MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit) : thread_soft_limit(soft_limit) , thread_hard_limit(hard_limit) @@ -77,16 +74,18 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac } const auto & empty_detail = getOrCreateSchedulerDetail(""); - GET_MIN_TSO_METRIC("", "min_tso")->Set(empty_detail.min_query_id.query_ts); - GET_MIN_TSO_METRIC("", "thread_soft_limit")->Set(thread_soft_limit); - GET_MIN_TSO_METRIC("", "thread_hard_limit")->Set(thread_hard_limit); - GET_MIN_TSO_METRIC("", "estimated_thread_usage")->Set(empty_detail.estimated_thread_usage); - GET_MIN_TSO_METRIC("", "global_estimated_thread_usage")->Set(global_estimated_thread_usage); - GET_MIN_TSO_METRIC("", "waiting_queries_count")->Set(0); - GET_MIN_TSO_METRIC("", "active_queries_count")->Set(0); - GET_MIN_TSO_METRIC("", "waiting_tasks_count")->Set(0); - GET_MIN_TSO_METRIC("", "active_tasks_count")->Set(0); - GET_MIN_TSO_METRIC("", "hard_limit_exceeded_count")->Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_min_tso, "").Set(empty_detail.min_query_id.query_ts); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_thread_soft_limit, "").Set(thread_soft_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_thread_hard_limit, "").Set(thread_hard_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, "") + .Set(empty_detail.estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_global_estimated_thread_usage, "") + .Set(global_estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_hard_limit_exceeded_count, "").Set(0); } } @@ -139,7 +138,11 @@ void MinTSOScheduler::deleteQuery( if (task != nullptr) task->scheduleThisTask(ScheduleState::FAILED); gather_it.second->waiting_tasks.pop(); - GET_MIN_TSO_METRIC(query_id.resource_group_name, "waiting_tasks_count")->Decrement(); + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group, + type_waiting_tasks_count, + query_id.resource_group_name) + .Decrement(); } } } @@ -164,8 +167,10 @@ void MinTSOScheduler::deleteQuery( detail.waiting_set.size()); detail.active_set.erase(query_id); detail.waiting_set.erase(query_id); - GET_MIN_TSO_METRIC(query_id.resource_group_name, "waiting_queries_count")->Set(detail.waiting_set.size()); - GET_MIN_TSO_METRIC(query_id.resource_group_name, "active_queries_count")->Set(detail.active_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, query_id.resource_group_name) + .Set(detail.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, query_id.resource_group_name) + .Set(detail.active_set.size()); /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. if (detail.updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it", log)) @@ -204,8 +209,9 @@ void MinTSOScheduler::releaseThreadsThenSchedule( updated_estimated_threads); detail.estimated_thread_usage = updated_estimated_threads; - GET_MIN_TSO_METRIC(resource_group_name, "estimated_thread_usage")->Set(detail.estimated_thread_usage); - GET_MIN_TSO_METRIC(resource_group_name, "active_tasks_count")->Decrement(); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, resource_group_name) + .Set(detail.estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, resource_group_name).Decrement(); /// as tasks release some threads, so some tasks would get scheduled. scheduleWaitingQueries(detail, task_manager, log); if (detail.active_set.size() + detail.waiting_set.size() == 0) @@ -228,8 +234,10 @@ void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskMa detail.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); detail.active_set.erase(current_query_id); detail.waiting_set.erase(current_query_id); - GET_MIN_TSO_METRIC(detail.resource_group_name, "waiting_queries_count")->Set(detail.waiting_set.size()); - GET_MIN_TSO_METRIC(detail.resource_group_name, "active_queries_count")->Set(detail.active_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, detail.resource_group_name) + .Set(detail.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, detail.resource_group_name) + .Set(detail.active_set.size()); continue; } @@ -253,7 +261,11 @@ void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskMa if (task != nullptr) task->getScheduleEntry().schedule(ScheduleState::EXCEEDED); gather_set.second->waiting_tasks.pop(); - GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Decrement(); + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group, + type_waiting_tasks_count, + detail.resource_group_name) + .Decrement(); continue; } @@ -265,17 +277,19 @@ void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskMa return; } gather_set.second->waiting_tasks.pop(); - GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Decrement(); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, detail.resource_group_name) + .Decrement(); } } LOG_DEBUG( log, "query {} (is min = {}) is scheduled from waiting set (size = {}).", current_query_id.toString(), - current_query_id == min_query_id, - waiting_set.size()); - waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active - GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); + current_query_id == detail.min_query_id, + detail.waiting_set.size()); + detail.waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, detail.resource_group_name) + .Set(detail.waiting_set.size()); } } @@ -303,11 +317,15 @@ bool MinTSOScheduler::scheduleImp( { detail.estimated_thread_usage += needed_threads; global_estimated_thread_usage += needed_threads; - GET_MIN_TSO_METRIC(detail.resource_group_name, "active_tasks_count")->Increment(); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, detail.resource_group_name) + .Increment(); } - GET_MIN_TSO_METRIC(detail.resource_group_name, "active_queries_count")->Set(detail.active_set.size()); - GET_MIN_TSO_METRIC(detail.resource_group_name, "estimated_thread_usage")->Set(detail.estimated_thread_usage); - GET_MIN_TSO_METRIC("", "global_estimated_thread_usage")->Set(global_estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, detail.resource_group_name) + .Set(detail.active_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, detail.resource_group_name) + .Set(detail.estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_global_estimated_thread_usage, "") + .Set(global_estimated_thread_usage); LOG_DEBUG( log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} " @@ -332,16 +350,20 @@ bool MinTSOScheduler::scheduleImp( "threads are unavailable for the query {} ({} min_query_id {}) {}, need {}, but used {} of the thread " "hard limit {}, {} active and {} waiting queries.", query_id.toString(), - query_id == min_query_id ? "is" : "is newer than", - min_query_id.toString(), + query_id == detail.min_query_id ? "is" : "is newer than", + detail.min_query_id.toString(), isWaiting ? "from the waiting set" : "when directly schedule it", needed_threads, - estimated_thread_usage, + detail.estimated_thread_usage, thread_hard_limit, - active_set.size(), - waiting_set.size()); + detail.active_set.size(), + detail.waiting_set.size()); LOG_ERROR(log, "{}", msg); - GET_METRIC(tiflash_task_scheduler, type_hard_limit_exceeded_count).Increment(); + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group, + type_hard_limit_exceeded_count, + detail.resource_group_name) + .Increment(); if (isWaiting) { /// set this task be failed to schedule, and the task will throw exception. @@ -357,8 +379,10 @@ bool MinTSOScheduler::scheduleImp( { detail.waiting_set.insert(query_id); query_task_set->waiting_tasks.push(schedule_entry.getMPPTaskId()); - GET_MIN_TSO_METRIC(detail.resource_group_name, "waiting_queries_count")->Set(detail.waiting_set.size()); - GET_MIN_TSO_METRIC(detail.resource_group_name, "waiting_tasks_count")->Increment(); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, detail.resource_group_name) + .Set(detail.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, detail.resource_group_name) + .Increment(); } LOG_INFO( log, @@ -402,8 +426,8 @@ bool MinTSOScheduler::SchedulerDetail::updateMinQueryId( if (min_query_id != old_min_query_id) /// if min_query_id == MPPTaskId::Max_Query_Id and the query_id is not to be cancelled, the used_threads, active_set.size() and waiting_set.size() must be 0. { - GET_MIN_TSO_METRIC(resource_group_name, "min_tso") - ->Set(min_query_id.query_ts == 0 ? min_query_id.start_ts : min_query_id.query_ts); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_min_tso, resource_group_name) + .Set(min_query_id.query_ts == 0 ? min_query_id.start_ts : min_query_id.query_ts); LOG_DEBUG( log, "min_query_id query is updated from {} to {} {}, used threads = {}, {} active and {} waiting queries.", From 3b66abde319770d8af5785998a8e8dfeb6bd5195 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 11:03:44 +0800 Subject: [PATCH 03/20] detail -> entry Signed-off-by: guo-shaoge --- dbms/src/Common/TiFlashMetrics.cpp | 14 --- dbms/src/Common/TiFlashMetrics.h | 5 - dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 165 ++++++++++++------------- dbms/src/Flash/Mpp/MinTSOScheduler.h | 12 +- 4 files changed, 87 insertions(+), 109 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.cpp b/dbms/src/Common/TiFlashMetrics.cpp index 258d3e3efaa..28d9b5b949c 100644 --- a/dbms/src/Common/TiFlashMetrics.cpp +++ b/dbms/src/Common/TiFlashMetrics.cpp @@ -58,20 +58,6 @@ TiFlashMetrics::TiFlashMetrics() .Name("tiflash_storage_sync_replica_ru") .Help("RU for synchronous replica of keyspace") .Register(*registry); - registered_min_tso_scheduler_family - = &prometheus::BuildGauge().Name("tiflash_task_scheduler").Help("Min-tso task scheduler").Register(*registry); -} - -prometheus::Gauge * TiFlashMetrics::getOrCreateMinTSOGauge(const String & resource_group_name, const String & type) -{ - const String key = resource_group_name + type; - std::map labels{{"resource_group", resource_group_name}, {"type", type}}; - auto iter = registered_resource_group_min_tso.find(key); - if (iter != registered_resource_group_min_tso.end()) - { - return iter->second; - } - return registered_resource_group_min_tso[key] = ®istered_min_tso_scheduler_family->Add(labels); } void TiFlashMetrics::addReplicaSyncRU(UInt32 keyspace_id, UInt64 ru) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 0787cf9bea3..b5a2e3db2c3 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -890,8 +890,6 @@ class TiFlashMetrics void addReplicaSyncRU(UInt32 keyspace_id, UInt64 ru); - prometheus::Gauge * getOrCreateMinTSOGauge(const String & resource_group_name, const String & type); - private: TiFlashMetrics(); @@ -921,9 +919,6 @@ class TiFlashMetrics std::mutex replica_sync_ru_mtx; std::unordered_map registered_keyspace_sync_replica_ru; - prometheus::Family * registered_min_tso_scheduler_family; - std::unordered_map registered_resource_group_min_tso; - public: #define MAKE_METRIC_MEMBER_M(family_name, help, type, ...) \ MetricFamily family_name \ diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 36cbc33d42a..5168ac26c41 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -73,12 +73,12 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac active_set_soft_limit); } - const auto & empty_detail = getOrCreateSchedulerDetail(""); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_min_tso, "").Set(empty_detail.min_query_id.query_ts); + const auto & empty_entry = getOrCreateGroupEntry(""); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_min_tso, "").Set(empty_entry.min_query_id.query_ts); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_thread_soft_limit, "").Set(thread_soft_limit); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_thread_hard_limit, "").Set(thread_hard_limit); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, "") - .Set(empty_detail.estimated_thread_usage); + .Set(empty_entry.estimated_thread_usage); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_global_estimated_thread_usage, "") .Set(global_estimated_thread_usage); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, "").Set(0); @@ -120,7 +120,7 @@ void MinTSOScheduler::deleteQuery( return; } - auto & detail = mustGetSchedulerDetail(query_id.resource_group_name); + auto & entry = mustGetGroupEntry(query_id.resource_group_name); bool all_gathers_deleted = true; auto query = task_manager.getMPPQueryWithoutLock(query_id); @@ -160,22 +160,22 @@ void MinTSOScheduler::deleteQuery( "{} query {} (is min = {}) is deleted from active set {} left {} or waiting set {} left {}.", is_cancelled ? "Cancelled" : "Finished", query_id.toString(), - query_id == detail.min_query_id, - detail.active_set.find(query_id) != detail.active_set.end(), - detail.active_set.size(), - detail.waiting_set.find(query_id) != detail.waiting_set.end(), - detail.waiting_set.size()); - detail.active_set.erase(query_id); - detail.waiting_set.erase(query_id); + query_id == entry.min_query_id, + entry.active_set.find(query_id) != entry.active_set.end(), + entry.active_set.size(), + entry.waiting_set.find(query_id) != entry.waiting_set.end(), + entry.waiting_set.size()); + entry.active_set.erase(query_id); + entry.waiting_set.erase(query_id); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, query_id.resource_group_name) - .Set(detail.waiting_set.size()); + .Set(entry.waiting_set.size()); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, query_id.resource_group_name) - .Set(detail.active_set.size()); + .Set(entry.active_set.size()); /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. - if (detail.updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it", log)) + if (entry.updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it", log)) { - scheduleWaitingQueries(detail, task_manager, log); + scheduleWaitingQueries(entry, task_manager, log); } } else @@ -200,44 +200,44 @@ void MinTSOScheduler::releaseThreadsThenSchedule( return; } - auto & detail = mustGetSchedulerDetail(resource_group_name); - auto updated_estimated_threads = static_cast(detail.estimated_thread_usage) - needed_threads; + auto & entry = mustGetGroupEntry(resource_group_name); + auto updated_estimated_threads = static_cast(entry.estimated_thread_usage) - needed_threads; RUNTIME_ASSERT( updated_estimated_threads >= 0, log, "estimated_thread_usage should not be smaller than 0, actually is {}.", updated_estimated_threads); - detail.estimated_thread_usage = updated_estimated_threads; + entry.estimated_thread_usage = updated_estimated_threads; GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, resource_group_name) - .Set(detail.estimated_thread_usage); + .Set(entry.estimated_thread_usage); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, resource_group_name).Decrement(); /// as tasks release some threads, so some tasks would get scheduled. - scheduleWaitingQueries(detail, task_manager, log); - if (detail.active_set.size() + detail.waiting_set.size() == 0) + scheduleWaitingQueries(entry, task_manager, log); + if (entry.active_set.size() + entry.waiting_set.size() == 0) { - LOG_INFO(log, "min tso scheduler_detail of resouce group {} deleted", resource_group_name); - scheduler_details.erase(resource_group_name); + LOG_INFO(log, "min tso scheduler_entry of resouce group {} deleted", resource_group_name); + scheduler_entries.erase(resource_group_name); } } -void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskManager & task_manager, LoggerPtr log) +void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager & task_manager, LoggerPtr log) { /// schedule new tasks - while (!detail.waiting_set.empty()) + while (!entry.waiting_set.empty()) { - auto current_query_id = *detail.waiting_set.begin(); + auto current_query_id = *entry.waiting_set.begin(); auto query = task_manager.getMPPQueryWithoutLock(current_query_id); if (nullptr == query) /// silently solve this rare case { LOG_ERROR(log, "the waiting query {} is not in the task manager.", current_query_id.toString()); - detail.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); - detail.active_set.erase(current_query_id); - detail.waiting_set.erase(current_query_id); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, detail.resource_group_name) - .Set(detail.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, detail.resource_group_name) - .Set(detail.active_set.size()); + entry.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); + entry.active_set.erase(current_query_id); + entry.waiting_set.erase(current_query_id); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) + .Set(entry.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, entry.resource_group_name) + .Set(entry.active_set.size()); continue; } @@ -245,8 +245,8 @@ void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskMa log, "query {} (is min = {}) is to be scheduled from waiting set (size = {}).", current_query_id.toString(), - current_query_id == detail.min_query_id, - detail.waiting_set.size()); + current_query_id == entry.min_query_id, + entry.waiting_set.size()); /// schedule tasks one by one for (auto & gather_set : query->mpp_gathers) { @@ -264,7 +264,7 @@ void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskMa GET_RESOURCE_GROUP_METRIC( tiflash_resource_group, type_waiting_tasks_count, - detail.resource_group_name) + entry.resource_group_name) .Decrement(); continue; } @@ -277,7 +277,7 @@ void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskMa return; } gather_set.second->waiting_tasks.pop(); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, detail.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, entry.resource_group_name) .Decrement(); } } @@ -285,11 +285,11 @@ void MinTSOScheduler::scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskMa log, "query {} (is min = {}) is scheduled from waiting set (size = {}).", current_query_id.toString(), - current_query_id == detail.min_query_id, - detail.waiting_set.size()); - detail.waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, detail.resource_group_name) - .Set(detail.waiting_set.size()); + current_query_id == entry.min_query_id, + entry.waiting_set.size()); + entry.waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) + .Set(entry.waiting_set.size()); } } @@ -302,28 +302,28 @@ bool MinTSOScheduler::scheduleImp( bool & has_error) { auto needed_threads = schedule_entry.getNeededThreads(); - auto & detail = getOrCreateSchedulerDetail(query_id.resource_group_name); + auto & entry = getOrCreateGroupEntry(query_id.resource_group_name); auto check_for_new_min_tso - = query_id <= detail.min_query_id && global_estimated_thread_usage + needed_threads <= thread_hard_limit; - auto check_for_not_min_tso = (detail.active_set.size() < active_set_soft_limit - || detail.active_set.find(query_id) != detail.active_set.end()) - && (detail.estimated_thread_usage + needed_threads <= thread_soft_limit); + = query_id <= entry.min_query_id && global_estimated_thread_usage + needed_threads <= thread_hard_limit; + auto check_for_not_min_tso + = (entry.active_set.size() < active_set_soft_limit || entry.active_set.find(query_id) != entry.active_set.end()) + && (entry.estimated_thread_usage + needed_threads <= thread_soft_limit); if (check_for_new_min_tso || check_for_not_min_tso) { - detail.updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it", log); - detail.active_set.insert(query_id); + entry.updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it", log); + entry.active_set.insert(query_id); if (schedule_entry.schedule(ScheduleState::SCHEDULED)) { - detail.estimated_thread_usage += needed_threads; + entry.estimated_thread_usage += needed_threads; global_estimated_thread_usage += needed_threads; - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, detail.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, entry.resource_group_name) .Increment(); } - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, detail.resource_group_name) - .Set(detail.active_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, detail.resource_group_name) - .Set(detail.estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, entry.resource_group_name) + .Set(entry.active_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, entry.resource_group_name) + .Set(entry.estimated_thread_usage); GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_global_estimated_thread_usage, "") .Set(global_estimated_thread_usage); LOG_DEBUG( @@ -331,17 +331,17 @@ bool MinTSOScheduler::scheduleImp( "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} " "of the thread {} limit {}.", schedule_entry.getMPPTaskId().toString(), - detail.active_set.size(), + entry.active_set.size(), isWaiting ? "from the waiting set" : "directly", needed_threads, - detail.estimated_thread_usage, - detail.min_query_id == query_id ? "hard" : "soft", - detail.min_query_id == query_id ? thread_hard_limit : thread_soft_limit); + entry.estimated_thread_usage, + entry.min_query_id == query_id ? "hard" : "soft", + entry.min_query_id == query_id ? thread_hard_limit : thread_soft_limit); return true; } else { - bool is_query_id_min = query_id <= detail.min_query_id; + bool is_query_id_min = query_id <= entry.min_query_id; fiu_do_on(FailPoints::random_min_tso_scheduler_failpoint, is_query_id_min = true;); if (is_query_id_min) /// the min_query_id query should fully run, otherwise throw errors here. { @@ -350,19 +350,16 @@ bool MinTSOScheduler::scheduleImp( "threads are unavailable for the query {} ({} min_query_id {}) {}, need {}, but used {} of the thread " "hard limit {}, {} active and {} waiting queries.", query_id.toString(), - query_id == detail.min_query_id ? "is" : "is newer than", - detail.min_query_id.toString(), + query_id == entry.min_query_id ? "is" : "is newer than", + entry.min_query_id.toString(), isWaiting ? "from the waiting set" : "when directly schedule it", needed_threads, - detail.estimated_thread_usage, + entry.estimated_thread_usage, thread_hard_limit, - detail.active_set.size(), - detail.waiting_set.size()); + entry.active_set.size(), + entry.waiting_set.size()); LOG_ERROR(log, "{}", msg); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group, - type_hard_limit_exceeded_count, - detail.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_hard_limit_exceeded_count, entry.resource_group_name) .Increment(); if (isWaiting) { @@ -377,11 +374,11 @@ bool MinTSOScheduler::scheduleImp( } if (!isWaiting) { - detail.waiting_set.insert(query_id); + entry.waiting_set.insert(query_id); query_task_set->waiting_tasks.push(schedule_entry.getMPPTaskId()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, detail.resource_group_name) - .Set(detail.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, detail.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) + .Set(entry.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, entry.resource_group_name) .Increment(); } LOG_INFO( @@ -391,16 +388,16 @@ bool MinTSOScheduler::scheduleImp( "required threads count are {}, waiting set size = {}", query_id.toString(), !isWaiting, - thread_soft_limit - detail.estimated_thread_usage, - active_set_soft_limit - detail.active_set.size(), + thread_soft_limit - entry.estimated_thread_usage, + active_set_soft_limit - entry.active_set.size(), needed_threads, - detail.waiting_set.size()); + entry.waiting_set.size()); return false; } } /// if return true, then need to schedule the waiting tasks of the min_query_id. -bool MinTSOScheduler::SchedulerDetail::updateMinQueryId( +bool MinTSOScheduler::GroupEntry::updateMinQueryId( const MPPQueryId & query_id, bool retired, const String & msg, @@ -441,20 +438,20 @@ bool MinTSOScheduler::SchedulerDetail::updateMinQueryId( return force_scheduling; } -MinTSOScheduler::SchedulerDetail & MinTSOScheduler::mustGetSchedulerDetail(const String & resource_group_name) +MinTSOScheduler::GroupEntry & MinTSOScheduler::mustGetGroupEntry(const String & resource_group_name) { - auto iter = scheduler_details.find(resource_group_name); - RUNTIME_CHECK(iter != scheduler_details.end()); + auto iter = scheduler_entries.find(resource_group_name); + RUNTIME_CHECK(iter != scheduler_entries.end()); return iter->second; } -MinTSOScheduler::SchedulerDetail & MinTSOScheduler::getOrCreateSchedulerDetail(const String & resource_group_name) +MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const String & resource_group_name) { const String name_with_prefix = "rg_" + resource_group_name; - auto iter = scheduler_details.find(resource_group_name); - if (iter == scheduler_details.end()) + auto iter = scheduler_entries.find(resource_group_name); + if (iter == scheduler_entries.end()) { - iter = scheduler_details.insert({resource_group_name, SchedulerDetail(resource_group_name)}).first; + iter = scheduler_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; } return iter->second; } diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.h b/dbms/src/Flash/Mpp/MinTSOScheduler.h index cf44f6cbe4e..eca9c1acdea 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.h +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.h @@ -60,9 +60,9 @@ class MinTSOScheduler : private boost::noncopyable bool & has_error); bool isDisabled() const { return thread_hard_limit == 0 && thread_soft_limit == 0; } - struct SchedulerDetail + struct GroupEntry { - explicit SchedulerDetail(const String & name) + explicit GroupEntry(const String & name) : resource_group_name(name) , min_query_id(MPPTaskId::Max_Query_Id) , estimated_thread_usage(0) @@ -77,11 +77,11 @@ class MinTSOScheduler : private boost::noncopyable bool updateMinQueryId(const MPPQueryId & query_id, bool retired, const String & msg, LoggerPtr log); }; - void scheduleWaitingQueries(SchedulerDetail & detail, MPPTaskManager & task_manager, LoggerPtr log); - SchedulerDetail & mustGetSchedulerDetail(const String & resource_group_name); - SchedulerDetail & getOrCreateSchedulerDetail(const String & resource_group_name); + void scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager & task_manager, LoggerPtr log); + GroupEntry & mustGetGroupEntry(const String & resource_group_name); + GroupEntry & getOrCreateGroupEntry(const String & resource_group_name); - std::unordered_map scheduler_details; + std::unordered_map scheduler_entries; UInt64 thread_soft_limit; UInt64 thread_hard_limit; UInt64 global_estimated_thread_usage; From 8da9c625f210700dd6464a56cc6a37eb2fbced12 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 11:23:05 +0800 Subject: [PATCH 04/20] case; refine Signed-off-by: guo-shaoge --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 9 +- dbms/src/Flash/Mpp/MinTSOScheduler.h | 2 +- dbms/src/Flash/tests/gtest_compute_server.cpp | 132 +++++++++++++++--- 3 files changed, 115 insertions(+), 28 deletions(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 5168ac26c41..271783affb1 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -27,11 +27,11 @@ extern const char random_min_tso_scheduler_failpoint[]; constexpr UInt64 OS_THREAD_SOFT_LIMIT = 100000; -MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit) +MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit_) : thread_soft_limit(soft_limit) , thread_hard_limit(hard_limit) , global_estimated_thread_usage(0) - , active_set_soft_limit(active_set_soft_limit) + , active_set_soft_limit(active_set_soft_limit_) , log(Logger::get()) { auto cores = static_cast(getNumberOfLogicalCPUCores() / 2); @@ -167,9 +167,9 @@ void MinTSOScheduler::deleteQuery( entry.waiting_set.size()); entry.active_set.erase(query_id); entry.waiting_set.erase(query_id); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, query_id.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, query_id.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, entry.resource_group_name) .Set(entry.active_set.size()); /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. @@ -447,7 +447,6 @@ MinTSOScheduler::GroupEntry & MinTSOScheduler::mustGetGroupEntry(const String & MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const String & resource_group_name) { - const String name_with_prefix = "rg_" + resource_group_name; auto iter = scheduler_entries.find(resource_group_name); if (iter == scheduler_entries.end()) { diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.h b/dbms/src/Flash/Mpp/MinTSOScheduler.h index eca9c1acdea..85ff2d177cc 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.h +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.h @@ -35,7 +35,7 @@ using MPPGatherTaskSetPtr = std::shared_ptr; class MinTSOScheduler : private boost::noncopyable { public: - MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit); + MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit_); ~MinTSOScheduler() = default; /// try to schedule this task if it is the min_query_id query or there are enough threads, otherwise put it into the waiting set. /// NOTE: call tryToSchedule under the lock protection of MPPTaskManager diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index ca60814a064..f574e5927a5 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -945,8 +945,16 @@ try context.context->setSetting("task_scheduler_active_set_soft_limit", active_set_soft_limit); startServers(1); setCancelTest(); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 0); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); std::vector running_queries; std::vector gather_ids; try @@ -958,12 +966,28 @@ try } using namespace std::literals::chrono_literals; std::this_thread::sleep_for(2s); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 2); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); addOneQuery(1, running_queries, gather_ids); std::this_thread::sleep_for(2s); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 3); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 3); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); for (const auto & gather_id : gather_ids) MockComputeServerManager::instance().cancelGather(gather_id); for (auto & t : running_queries) @@ -977,18 +1001,42 @@ try } using namespace std::literals::chrono_literals; std::this_thread::sleep_for(2s); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 2); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); addOneQuery(30, running_queries, gather_ids); std::this_thread::sleep_for(2s); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 1); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 2); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 1); /// cancel 1 running query MockComputeServerManager::instance().cancelGather(gather_ids[0]); running_queries[0].join(); std::this_thread::sleep_for(2s); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 2); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); for (size_t i = 1; i < running_queries.size(); i++) MockComputeServerManager::instance().cancelGather(gather_ids[i]); for (size_t i = 1; i < running_queries.size(); i++) @@ -1011,8 +1059,16 @@ try { startServers(1); setCancelTest(); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 0); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); std::vector running_queries; std::vector gather_ids; auto multiple_gathers_properties = DB::tests::getDAGPropertiesForTest(serverNum(), 1, 1, 1); @@ -1029,8 +1085,16 @@ try using namespace std::literals::chrono_literals; std::this_thread::sleep_for(2s); /// 6 gathers, but two query - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 2); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); std::vector killed_gathers{0, 2, 4}; std::vector remaining_gathers{1, 3}; for (const auto i : killed_gathers) @@ -1044,22 +1108,46 @@ try assertGatherActive(gather_ids[i]); } /// the active query count should not change - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 2); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 2); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); /// kill single gather query MockComputeServerManager::instance().cancelGather(gather_ids[5]); assertGatherCancelled(gather_ids[5]); /// the active query count should be 1 - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 1); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 1); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); /// kill the rest gathers for (const auto i : remaining_gathers) { MockComputeServerManager::instance().cancelGather(gather_ids[i]); assertGatherCancelled(gather_ids[i]); } - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "active_queries_count")->Value() == 0); - ASSERT_TRUE(TiFlashMetrics::instance().getOrCreateMinTSOGauge("", "waiting_queries_count")->Value() == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .Value() + == 0); + ASSERT_TRUE( + TiFlashMetrics::instance() + .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .Value() + == 0); for (auto & t : running_queries) if (t.joinable()) t.join(); From 2a78ab996ccb7371362489e21c61cb2dfe1a4d38 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 11:40:17 +0800 Subject: [PATCH 05/20] rename tso metric name Signed-off-by: guo-shaoge --- dbms/src/Common/TiFlashMetrics.h | 5 +- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 96 +++++++++++++------ dbms/src/Flash/tests/gtest_compute_server.cpp | 66 ++++++++----- 3 files changed, 116 insertions(+), 51 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index b5a2e3db2c3..b68c050e7b1 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -694,7 +694,10 @@ namespace DB F(type_total_consumption, {"type", "total_consumption"}), \ F(type_bucket_fill_rate, {"type", "bucket_fill_rate"}), \ F(type_bucket_capacity, {"type", "bucket_capacity"}), \ - F(type_fetch_tokens_from_gac_count, {"type", "fetch_tokens_from_gac_count"}), \ + F(type_fetch_tokens_from_gac_count, {"type", "fetch_tokens_from_gac_count"})) \ + M(tiflash_resource_group_task_scheduler, \ + "min tso task scheduler of resource groups", \ + Gauge, \ F(type_min_tso, {"type", "min_tso"}), \ F(type_waiting_queries_count, {"type", "waiting_queries_count"}), \ F(type_active_queries_count, {"type", "active_queries_count"}), \ diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 271783affb1..bfdfbcbc6f6 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -74,18 +74,21 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac } const auto & empty_entry = getOrCreateGroupEntry(""); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_min_tso, "").Set(empty_entry.min_query_id.query_ts); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_thread_soft_limit, "").Set(thread_soft_limit); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_thread_hard_limit, "").Set(thread_hard_limit); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, "") + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_min_tso, "") + .Set(empty_entry.min_query_id.query_ts); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_thread_soft_limit, "") + .Set(thread_soft_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_thread_hard_limit, "") + .Set(thread_hard_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_estimated_thread_usage, "") .Set(empty_entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_global_estimated_thread_usage, "") + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_global_estimated_thread_usage, "") .Set(global_estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_hard_limit_exceeded_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_waiting_queries_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_active_queries_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_waiting_tasks_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_active_tasks_count, "").Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_hard_limit_exceeded_count, "").Set(0); } } @@ -139,7 +142,7 @@ void MinTSOScheduler::deleteQuery( task->scheduleThisTask(ScheduleState::FAILED); gather_it.second->waiting_tasks.pop(); GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group, + tiflash_resource_group_task_scheduler, type_waiting_tasks_count, query_id.resource_group_name) .Decrement(); @@ -167,9 +170,15 @@ void MinTSOScheduler::deleteQuery( entry.waiting_set.size()); entry.active_set.erase(query_id); entry.waiting_set.erase(query_id); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_waiting_queries_count, + entry.resource_group_name) .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_active_queries_count, + entry.resource_group_name) .Set(entry.active_set.size()); /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. @@ -209,9 +218,10 @@ void MinTSOScheduler::releaseThreadsThenSchedule( updated_estimated_threads); entry.estimated_thread_usage = updated_estimated_threads; - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_estimated_thread_usage, resource_group_name) .Set(entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, resource_group_name).Decrement(); + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_active_tasks_count, resource_group_name) + .Decrement(); /// as tasks release some threads, so some tasks would get scheduled. scheduleWaitingQueries(entry, task_manager, log); if (entry.active_set.size() + entry.waiting_set.size() == 0) @@ -234,9 +244,15 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager entry.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); entry.active_set.erase(current_query_id); entry.waiting_set.erase(current_query_id); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_waiting_queries_count, + entry.resource_group_name) .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_active_queries_count, + entry.resource_group_name) .Set(entry.active_set.size()); continue; } @@ -262,7 +278,7 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager task->getScheduleEntry().schedule(ScheduleState::EXCEEDED); gather_set.second->waiting_tasks.pop(); GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group, + tiflash_resource_group_task_scheduler, type_waiting_tasks_count, entry.resource_group_name) .Decrement(); @@ -277,7 +293,10 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager return; } gather_set.second->waiting_tasks.pop(); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_waiting_tasks_count, + entry.resource_group_name) .Decrement(); } } @@ -288,7 +307,10 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager current_query_id == entry.min_query_id, entry.waiting_set.size()); entry.waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_waiting_queries_count, + entry.resource_group_name) .Set(entry.waiting_set.size()); } } @@ -317,14 +339,23 @@ bool MinTSOScheduler::scheduleImp( { entry.estimated_thread_usage += needed_threads; global_estimated_thread_usage += needed_threads; - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_tasks_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_active_tasks_count, + entry.resource_group_name) .Increment(); } - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_active_queries_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_active_queries_count, + entry.resource_group_name) .Set(entry.active_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_estimated_thread_usage, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_estimated_thread_usage, + entry.resource_group_name) .Set(entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_global_estimated_thread_usage, "") + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_global_estimated_thread_usage, "") .Set(global_estimated_thread_usage); LOG_DEBUG( log, @@ -359,7 +390,10 @@ bool MinTSOScheduler::scheduleImp( entry.active_set.size(), entry.waiting_set.size()); LOG_ERROR(log, "{}", msg); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_hard_limit_exceeded_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_hard_limit_exceeded_count, + entry.resource_group_name) .Increment(); if (isWaiting) { @@ -376,9 +410,15 @@ bool MinTSOScheduler::scheduleImp( { entry.waiting_set.insert(query_id); query_task_set->waiting_tasks.push(schedule_entry.getMPPTaskId()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_queries_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_waiting_queries_count, + entry.resource_group_name) .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_waiting_tasks_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_resource_group_task_scheduler, + type_waiting_tasks_count, + entry.resource_group_name) .Increment(); } LOG_INFO( @@ -423,7 +463,7 @@ bool MinTSOScheduler::GroupEntry::updateMinQueryId( if (min_query_id != old_min_query_id) /// if min_query_id == MPPTaskId::Max_Query_Id and the query_id is not to be cancelled, the used_threads, active_set.size() and waiting_set.size() must be 0. { - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group, type_min_tso, resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_min_tso, resource_group_name) .Set(min_query_id.query_ts == 0 ? min_query_id.start_ts : min_query_id.query_ts); LOG_DEBUG( log, diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index f574e5927a5..7f4dfa63441 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -947,12 +947,14 @@ try setCancelTest(); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 0); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); std::vector running_queries; @@ -968,24 +970,28 @@ try std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); addOneQuery(1, running_queries, gather_ids); std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 3); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); for (const auto & gather_id : gather_ids) @@ -1003,24 +1009,28 @@ try std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); addOneQuery(30, running_queries, gather_ids); std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 1); /// cancel 1 running query @@ -1029,12 +1039,14 @@ try std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); for (size_t i = 1; i < running_queries.size(); i++) @@ -1061,12 +1073,14 @@ try setCancelTest(); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 0); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); std::vector running_queries; @@ -1087,12 +1101,14 @@ try /// 6 gathers, but two query ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); std::vector killed_gathers{0, 2, 4}; @@ -1110,12 +1126,14 @@ try /// the active query count should not change ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); /// kill single gather query @@ -1124,12 +1142,14 @@ try /// the active query count should be 1 ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 1); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); /// kill the rest gathers @@ -1140,12 +1160,14 @@ try } ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_active_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") .Value() == 0); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group.get(tiflash_resource_group_metrics::type_waiting_queries_count, "") + .tiflash_resource_group_task_scheduler + .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); for (auto & t : running_queries) From 3dd452b6e879439e9749721ec4320f0eb57307da Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 13:52:00 +0800 Subject: [PATCH 06/20] rename metric name Signed-off-by: guo-shaoge --- dbms/src/Common/TiFlashMetrics.h | 30 +++---- dbms/src/Flash/Executor/PipelineExecutor.cpp | 5 -- dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp | 3 +- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 90 ++++--------------- .../src/Flash/ResourceControl/TokenBucket.cpp | 8 +- dbms/src/TiDB/Etcd/Client.cpp | 5 +- metrics/grafana/tiflash_summary.json | 48 +++++----- 7 files changed, 67 insertions(+), 122 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index b68c050e7b1..47fdabbb9cc 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -468,6 +468,19 @@ namespace DB F(type_active_threads_of_dispatch_mpp, {"type", "rpc_dispatch_mpp"}), \ F(type_active_rpc_async_worker, {"type", "rpc_async_worker_active"}), \ F(type_total_rpc_async_worker, {"type", "rpc_async_worker_total"})) \ + M(tiflash_task_scheduler, \ + "Min-tso task scheduler", \ + Gauge, \ + F(type_min_tso, {"type", "min_tso"}), \ + F(type_waiting_queries_count, {"type", "waiting_queries_count"}), \ + F(type_active_queries_count, {"type", "active_queries_count"}), \ + F(type_waiting_tasks_count, {"type", "waiting_tasks_count"}), \ + F(type_active_tasks_count, {"type", "active_tasks_count"}), \ + F(type_global_estimated_thread_usage, {"type", "global_estimated_thread_usage"}), \ + F(type_estimated_thread_usage, {"type", "estimated_thread_usage"}), \ + F(type_thread_soft_limit, {"type", "thread_soft_limit"}), \ + F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ + F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) \ M(tiflash_task_scheduler_waiting_duration_seconds, \ "Bucketed histogram of task waiting for scheduling duration", \ Histogram, \ @@ -687,27 +700,14 @@ namespace DB F(type_fsync, {{"type", "fsync"}}, ExpBuckets{0.0001, 2, 20})) \ M(tiflash_storage_delta_index_cache, "", Counter, F(type_hit, {"type", "hit"}), F(type_miss, {"type", "miss"})) \ M(tiflash_resource_group, \ - "meta infos of resource groups", \ + "meta info of resource group", \ Gauge, \ F(type_remaining_tokens, {"type", "remaining_tokens"}), \ F(type_avg_speed, {"type", "avg_speed"}), \ F(type_total_consumption, {"type", "total_consumption"}), \ F(type_bucket_fill_rate, {"type", "bucket_fill_rate"}), \ F(type_bucket_capacity, {"type", "bucket_capacity"}), \ - F(type_fetch_tokens_from_gac_count, {"type", "fetch_tokens_from_gac_count"})) \ - M(tiflash_resource_group_task_scheduler, \ - "min tso task scheduler of resource groups", \ - Gauge, \ - F(type_min_tso, {"type", "min_tso"}), \ - F(type_waiting_queries_count, {"type", "waiting_queries_count"}), \ - F(type_active_queries_count, {"type", "active_queries_count"}), \ - F(type_waiting_tasks_count, {"type", "waiting_tasks_count"}), \ - F(type_active_tasks_count, {"type", "active_tasks_count"}), \ - F(type_estimated_thread_usage, {"type", "estimated_thread_usage"}), \ - F(type_global_estimated_thread_usage, {"type", "global_estimated_thread_usage"}), \ - F(type_thread_soft_limit, {"type", "thread_soft_limit"}), \ - F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ - F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) + F(type_fetch_tokens_from_gac_count, {"type", "fetch_tokens_from_gac_count"})) /// Buckets with boundaries [start * base^0, start * base^1, ..., start * base^(size-1)] diff --git a/dbms/src/Flash/Executor/PipelineExecutor.cpp b/dbms/src/Flash/Executor/PipelineExecutor.cpp index bf39ea51f68..5159008eb9c 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutor.cpp @@ -110,11 +110,6 @@ ExecutionResult PipelineExecutor::execute(ResultHandler && result_handler) } LOG_TRACE(log, "query finish with {}", exec_context.getQueryProfileInfo().toJson()); - // For read_ru, only report it to GAC for now. - LocalAdmissionController::global_instance->consumeResource( - exec_context.getResourceGroupName(), - dagContext().getReadRU(), - 0); return exec_context.toExecutionResult(); } diff --git a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp index e46135c79af..9c86bde57c0 100644 --- a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp @@ -63,7 +63,8 @@ void MPPTaskScheduleEntry::waitForSchedule() std::unique_lock lock(schedule_mu); schedule_cv.wait(lock, [&] { return schedule_state != ScheduleState::WAITING; }); time_cost = stopwatch.elapsedSeconds(); - GET_METRIC(tiflash_task_scheduler_waiting_duration_seconds).Observe(time_cost); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler_waiting_duration_seconds, getResourceGroupName()) + .Observe(time_cost); if (schedule_state == ScheduleState::EXCEEDED) { diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index bfdfbcbc6f6..0239e1407e2 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -72,23 +72,6 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac thread_soft_limit, active_set_soft_limit); } - - const auto & empty_entry = getOrCreateGroupEntry(""); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_min_tso, "") - .Set(empty_entry.min_query_id.query_ts); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_thread_soft_limit, "") - .Set(thread_soft_limit); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_thread_hard_limit, "") - .Set(thread_hard_limit); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_estimated_thread_usage, "") - .Set(empty_entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_global_estimated_thread_usage, "") - .Set(global_estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_waiting_queries_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_active_queries_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_waiting_tasks_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_active_tasks_count, "").Set(0); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_hard_limit_exceeded_count, "").Set(0); } } @@ -142,7 +125,7 @@ void MinTSOScheduler::deleteQuery( task->scheduleThisTask(ScheduleState::FAILED); gather_it.second->waiting_tasks.pop(); GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, + tiflash_task_scheduler, type_waiting_tasks_count, query_id.resource_group_name) .Decrement(); @@ -170,15 +153,9 @@ void MinTSOScheduler::deleteQuery( entry.waiting_set.size()); entry.active_set.erase(query_id); entry.waiting_set.erase(query_id); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_waiting_queries_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_active_queries_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, entry.resource_group_name) .Set(entry.active_set.size()); /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. @@ -218,10 +195,9 @@ void MinTSOScheduler::releaseThreadsThenSchedule( updated_estimated_threads); entry.estimated_thread_usage = updated_estimated_threads; - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_estimated_thread_usage, resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, resource_group_name) .Set(entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_active_tasks_count, resource_group_name) - .Decrement(); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, resource_group_name).Decrement(); /// as tasks release some threads, so some tasks would get scheduled. scheduleWaitingQueries(entry, task_manager, log); if (entry.active_set.size() + entry.waiting_set.size() == 0) @@ -244,15 +220,9 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager entry.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); entry.active_set.erase(current_query_id); entry.waiting_set.erase(current_query_id); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_waiting_queries_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_active_queries_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, entry.resource_group_name) .Set(entry.active_set.size()); continue; } @@ -278,7 +248,7 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager task->getScheduleEntry().schedule(ScheduleState::EXCEEDED); gather_set.second->waiting_tasks.pop(); GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, + tiflash_task_scheduler, type_waiting_tasks_count, entry.resource_group_name) .Decrement(); @@ -293,10 +263,7 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager return; } gather_set.second->waiting_tasks.pop(); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_waiting_tasks_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_tasks_count, entry.resource_group_name) .Decrement(); } } @@ -307,10 +274,7 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager current_query_id == entry.min_query_id, entry.waiting_set.size()); entry.waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_waiting_queries_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) .Set(entry.waiting_set.size()); } } @@ -339,23 +303,14 @@ bool MinTSOScheduler::scheduleImp( { entry.estimated_thread_usage += needed_threads; global_estimated_thread_usage += needed_threads; - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_active_tasks_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, entry.resource_group_name) .Increment(); } - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_active_queries_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, entry.resource_group_name) .Set(entry.active_set.size()); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_estimated_thread_usage, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, entry.resource_group_name) .Set(entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_global_estimated_thread_usage, "") + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage, entry.resource_group_name) .Set(global_estimated_thread_usage); LOG_DEBUG( log, @@ -390,10 +345,7 @@ bool MinTSOScheduler::scheduleImp( entry.active_set.size(), entry.waiting_set.size()); LOG_ERROR(log, "{}", msg); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_hard_limit_exceeded_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_hard_limit_exceeded_count, entry.resource_group_name) .Increment(); if (isWaiting) { @@ -410,15 +362,9 @@ bool MinTSOScheduler::scheduleImp( { entry.waiting_set.insert(query_id); query_task_set->waiting_tasks.push(schedule_entry.getMPPTaskId()); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_waiting_queries_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC( - tiflash_resource_group_task_scheduler, - type_waiting_tasks_count, - entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_tasks_count, entry.resource_group_name) .Increment(); } LOG_INFO( @@ -463,7 +409,7 @@ bool MinTSOScheduler::GroupEntry::updateMinQueryId( if (min_query_id != old_min_query_id) /// if min_query_id == MPPTaskId::Max_Query_Id and the query_id is not to be cancelled, the used_threads, active_set.size() and waiting_set.size() must be 0. { - GET_RESOURCE_GROUP_METRIC(tiflash_resource_group_task_scheduler, type_min_tso, resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_min_tso, resource_group_name) .Set(min_query_id.query_ts == 0 ? min_query_id.start_ts : min_query_id.query_ts); LOG_DEBUG( log, diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.cpp b/dbms/src/Flash/ResourceControl/TokenBucket.cpp index badf826dfe7..07a5e2fea9d 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.cpp +++ b/dbms/src/Flash/ResourceControl/TokenBucket.cpp @@ -85,7 +85,13 @@ double TokenBucket::getAvgSpeedPerSec() void TokenBucket::compact(const TokenBucket::TimePoint & timepoint) { - tokens += getDynamicTokens(timepoint); + auto dynamic_tokens = getDynamicTokens(timepoint); + RUNTIME_CHECK(dynamic_tokens >= 0.0); + // To avoid getDynamicTokens() too frequently. + if (dynamic_tokens == 0.0) + return; + + tokens += dynamic_tokens; if (tokens >= capacity) tokens = capacity; last_compact_timepoint = timepoint; diff --git a/dbms/src/TiDB/Etcd/Client.cpp b/dbms/src/TiDB/Etcd/Client.cpp index d9ea6e3749e..a1461a9e78b 100644 --- a/dbms/src/TiDB/Etcd/Client.cpp +++ b/dbms/src/TiDB/Etcd/Client.cpp @@ -318,8 +318,6 @@ std::unordered_set Client::getExistsServerID() throw Exception("getExistsServerID failed, grpc error: {}", status.error_message()); std::unordered_set exists_server_ids; - FmtBuffer fmt_buf; - fmt_buf.fmtAppend("all existing server ids: "); for (const auto & kv : range_resp.kvs()) { String key = kv.key(); @@ -327,9 +325,8 @@ std::unordered_set Client::getExistsServerID() RUNTIME_CHECK(prefix == TIDB_SERVER_ID_ETCD_PATH); String server_id_str(key.begin() + TIDB_SERVER_ID_ETCD_PATH.size() + 1, key.end()); exists_server_ids.insert(std::stoi(server_id_str)); - fmt_buf.fmtAppend("{};", server_id_str); } - LOG_INFO(log, fmt_buf.toString()); + LOG_INFO(log, "existing server ids: {}", exists_server_ids.size()); return exists_server_ids; } diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index a2366c1fbe7..59f5add1ae5 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -4200,9 +4200,9 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"min_tso\"}) by (instance)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"min_tso\"}) by (instance, resource_group)", "interval": "", - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{resource_group}}", "queryType": "randomWalk", "refId": "A" } @@ -4302,27 +4302,27 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_soft_limit\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_soft_limit\"}) by (instance, type, resource_group)", "interval": "", - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"estimated_thread_usage\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"estimated_thread_usage\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "B" }, { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_hard_limit\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_hard_limit\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "C" } @@ -4422,18 +4422,18 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_queries_count\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_queries_count\"}) by (instance, type, resource_group)", "interval": "", - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_queries_count\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_queries_count\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "B" } @@ -4533,18 +4533,18 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_tasks_count\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_tasks_count\"}) by (instance, type, resource_group)", "interval": "", - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_tasks_count\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_tasks_count\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "B" } @@ -4644,10 +4644,10 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"hard_limit_exceeded_count\"}) by (instance, type)", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"hard_limit_exceeded_count\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{resource_group}}", "queryType": "randomWalk", "refId": "B" } @@ -4745,28 +4745,28 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.80, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le))", + "expr": "histogram_quantile(0.80, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", "hide": true, "interval": "", - "legendFormat": "{{instance}}-80", + "legendFormat": "{{instance}}-{{resource_group}}-80", "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, - "expr": "histogram_quantile(0.90, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le))", + "expr": "histogram_quantile(0.90, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", "hide": true, "interval": "", - "legendFormat": "{{instance}}-90", + "legendFormat": "{{instance}}-{{resource_group}}-90", "queryType": "randomWalk", "refId": "B" }, { "exemplar": true, - "expr": "histogram_quantile(1, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le))", + "expr": "histogram_quantile(1, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", "hide": false, "interval": "", - "legendFormat": "{{instance}}-100", + "legendFormat": "{{instance}}-{{resource_group}}-100", "queryType": "randomWalk", "refId": "C" } From e5cc3a55fb323c6260a6505bdb71f73d5b875142 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 14:08:53 +0800 Subject: [PATCH 07/20] fix case Signed-off-by: guo-shaoge --- dbms/src/Flash/tests/gtest_compute_server.cpp | 66 +++++++------------ 1 file changed, 22 insertions(+), 44 deletions(-) diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index 7f4dfa63441..e79c1d179fa 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -947,14 +947,12 @@ try setCancelTest(); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 0); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); std::vector running_queries; @@ -970,28 +968,24 @@ try std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); addOneQuery(1, running_queries, gather_ids); std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 3); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); for (const auto & gather_id : gather_ids) @@ -1009,28 +1003,24 @@ try std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); addOneQuery(30, running_queries, gather_ids); std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 1); /// cancel 1 running query @@ -1039,14 +1029,12 @@ try std::this_thread::sleep_for(2s); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); for (size_t i = 1; i < running_queries.size(); i++) @@ -1073,14 +1061,12 @@ try setCancelTest(); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 0); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); std::vector running_queries; @@ -1101,14 +1087,12 @@ try /// 6 gathers, but two query ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); std::vector killed_gathers{0, 2, 4}; @@ -1126,14 +1110,12 @@ try /// the active query count should not change ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 2); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); /// kill single gather query @@ -1142,14 +1124,12 @@ try /// the active query count should be 1 ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 1); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); /// kill the rest gathers @@ -1160,14 +1140,12 @@ try } ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_active_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_active_queries_count, "") .Value() == 0); ASSERT_TRUE( TiFlashMetrics::instance() - .tiflash_resource_group_task_scheduler - .get(tiflash_resource_group_task_scheduler_metrics::type_waiting_queries_count, "") + .tiflash_task_scheduler.get(tiflash_task_scheduler_metrics::type_waiting_queries_count, "") .Value() == 0); for (auto & t : running_queries) From e77b04ed943858272c863c78b4613427536dc3b8 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 15:57:58 +0800 Subject: [PATCH 08/20] refine Signed-off-by: guo-shaoge --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 0239e1407e2..8de7a0b14ab 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -73,6 +73,8 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac active_set_soft_limit); } } + GET_METRIC(tiflash_task_scheduler, type_thread_hard_limit).Set(thread_hard_limit); + GET_METRIC(tiflash_task_scheduler, type_thread_soft_limit).Set(thread_soft_limit); } bool MinTSOScheduler::tryToSchedule(MPPTaskScheduleEntry & schedule_entry, MPPTaskManager & task_manager) @@ -106,7 +108,7 @@ void MinTSOScheduler::deleteQuery( return; } - auto & entry = mustGetGroupEntry(query_id.resource_group_name); + auto & entry = getOrCreateGroupEntry(query_id.resource_group_name); bool all_gathers_deleted = true; auto query = task_manager.getMPPQueryWithoutLock(query_id); @@ -186,7 +188,7 @@ void MinTSOScheduler::releaseThreadsThenSchedule( return; } - auto & entry = mustGetGroupEntry(resource_group_name); + auto & entry = getOrCreateGroupEntry(resource_group_name); auto updated_estimated_threads = static_cast(entry.estimated_thread_usage) - needed_threads; RUNTIME_ASSERT( updated_estimated_threads >= 0, @@ -200,7 +202,7 @@ void MinTSOScheduler::releaseThreadsThenSchedule( GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, resource_group_name).Decrement(); /// as tasks release some threads, so some tasks would get scheduled. scheduleWaitingQueries(entry, task_manager, log); - if (entry.active_set.size() + entry.waiting_set.size() == 0) + if (entry.active_set.size() + entry.waiting_set.size() == 0 && entry.estimated_thread_usage == 0) { LOG_INFO(log, "min tso scheduler_entry of resouce group {} deleted", resource_group_name); scheduler_entries.erase(resource_group_name); @@ -310,8 +312,7 @@ bool MinTSOScheduler::scheduleImp( .Set(entry.active_set.size()); GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, entry.resource_group_name) .Set(entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage, entry.resource_group_name) - .Set(global_estimated_thread_usage); + GET_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage).Set(global_estimated_thread_usage); LOG_DEBUG( log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} " @@ -424,13 +425,6 @@ bool MinTSOScheduler::GroupEntry::updateMinQueryId( return force_scheduling; } -MinTSOScheduler::GroupEntry & MinTSOScheduler::mustGetGroupEntry(const String & resource_group_name) -{ - auto iter = scheduler_entries.find(resource_group_name); - RUNTIME_CHECK(iter != scheduler_entries.end()); - return iter->second; -} - MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const String & resource_group_name) { auto iter = scheduler_entries.find(resource_group_name); From 5aedfbd21fb7c88b9260deb99e757553bd9d6b29 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 16:27:50 +0800 Subject: [PATCH 09/20] refine Signed-off-by: guo-shaoge --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 6 +++--- metrics/grafana/tiflash_summary.json | 9 +++++++++ 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 8de7a0b14ab..fbf2ff3afef 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -73,8 +73,6 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac active_set_soft_limit); } } - GET_METRIC(tiflash_task_scheduler, type_thread_hard_limit).Set(thread_hard_limit); - GET_METRIC(tiflash_task_scheduler, type_thread_soft_limit).Set(thread_soft_limit); } bool MinTSOScheduler::tryToSchedule(MPPTaskScheduleEntry & schedule_entry, MPPTaskManager & task_manager) @@ -297,6 +295,8 @@ bool MinTSOScheduler::scheduleImp( auto check_for_not_min_tso = (entry.active_set.size() < active_set_soft_limit || entry.active_set.find(query_id) != entry.active_set.end()) && (entry.estimated_thread_usage + needed_threads <= thread_soft_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_hard_limit, entry.resource_group_name).Set(thread_hard_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, entry.resource_group_name).Set(thread_soft_limit); if (check_for_new_min_tso || check_for_not_min_tso) { entry.updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it", log); @@ -312,7 +312,7 @@ bool MinTSOScheduler::scheduleImp( .Set(entry.active_set.size()); GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, entry.resource_group_name) .Set(entry.estimated_thread_usage); - GET_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage).Set(global_estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage, entry.resource_group_name).Set(global_estimated_thread_usage); LOG_DEBUG( log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} " diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 59f5add1ae5..d240d23f39c 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -4325,6 +4325,15 @@ "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "C" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"global_estimated_thread_usage\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", + "refId": "C" } ], "thresholds": [], From c51248ff9589652de83c26d07f2791e46c814839 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 17:49:44 +0800 Subject: [PATCH 10/20] fix Signed-off-by: guo-shaoge --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index fbf2ff3afef..12530dbba65 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -186,6 +186,7 @@ void MinTSOScheduler::releaseThreadsThenSchedule( return; } + global_estimated_thread_usage -= needed_threads; auto & entry = getOrCreateGroupEntry(resource_group_name); auto updated_estimated_threads = static_cast(entry.estimated_thread_usage) - needed_threads; RUNTIME_ASSERT( @@ -295,8 +296,6 @@ bool MinTSOScheduler::scheduleImp( auto check_for_not_min_tso = (entry.active_set.size() < active_set_soft_limit || entry.active_set.find(query_id) != entry.active_set.end()) && (entry.estimated_thread_usage + needed_threads <= thread_soft_limit); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_hard_limit, entry.resource_group_name).Set(thread_hard_limit); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, entry.resource_group_name).Set(thread_soft_limit); if (check_for_new_min_tso || check_for_not_min_tso) { entry.updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it", log); @@ -312,7 +311,7 @@ bool MinTSOScheduler::scheduleImp( .Set(entry.active_set.size()); GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, entry.resource_group_name) .Set(entry.estimated_thread_usage); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage, entry.resource_group_name).Set(global_estimated_thread_usage); + GET_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage).Set(global_estimated_thread_usage); LOG_DEBUG( log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} " @@ -430,6 +429,8 @@ MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const Strin auto iter = scheduler_entries.find(resource_group_name); if (iter == scheduler_entries.end()) { + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_hard_limit, resource_group_name).Set(thread_hard_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, resource_group_name).Set(thread_soft_limit); iter = scheduler_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; } return iter->second; From fea6f0a219f9cfb727f34861cf1573400bc87044 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 14 Sep 2023 17:54:53 +0800 Subject: [PATCH 11/20] fmt Signed-off-by: guo-shaoge --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 12530dbba65..af299478e2b 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -429,8 +429,10 @@ MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const Strin auto iter = scheduler_entries.find(resource_group_name); if (iter == scheduler_entries.end()) { - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_hard_limit, resource_group_name).Set(thread_hard_limit); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, resource_group_name).Set(thread_soft_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_hard_limit, resource_group_name) + .Set(thread_hard_limit); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, resource_group_name) + .Set(thread_soft_limit); iter = scheduler_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; } return iter->second; From e6cf49266cb8e46fa14a68273980fed471e2654f Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 15 Sep 2023 14:09:22 +0800 Subject: [PATCH 12/20] unit test for TokenBucket::compact() Signed-off-by: guo-shaoge --- .../tests/gtest_resource_control_queue.cpp | 26 +++++++++++++++++++ .../src/Flash/ResourceControl/TokenBucket.cpp | 7 ++--- 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index de60a267e45..2f2bd0a7d5c 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -640,4 +640,30 @@ TEST_F(TestResourceControlQueue, cancel) } } +TEST_F(TestResourceControlQueue, tokenBucket) +{ + const double fill_rate = 10.0; + const double init_tokens = 10.0; + { + TokenBucket bucket(fill_rate, init_tokens, "log_id"); + for (int i = 0; i < 10; ++i) + { + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + bucket.peek(); + } + ASSERT_GT(bucket.peek(), init_tokens); + ASSERT_GE(bucket.peek(), init_tokens + fill_rate * 10 * std::chrono::milliseconds(10).count() / 1000); + } + { + TokenBucket bucket(fill_rate, init_tokens, "log_id"); + for (int i = 0; i < 10; ++i) + { + std::this_thread::sleep_for(std::chrono::microseconds(500)); + bucket.peek(); + } + ASSERT_GT(bucket.peek(), init_tokens); + ASSERT_GE(bucket.peek(), init_tokens + fill_rate * 10 * std::chrono::microseconds(500).count() / 1000000); + } +} + } // namespace DB::tests diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.cpp b/dbms/src/Flash/ResourceControl/TokenBucket.cpp index 07a5e2fea9d..72132d58c24 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.cpp +++ b/dbms/src/Flash/ResourceControl/TokenBucket.cpp @@ -87,7 +87,8 @@ void TokenBucket::compact(const TokenBucket::TimePoint & timepoint) { auto dynamic_tokens = getDynamicTokens(timepoint); RUNTIME_CHECK(dynamic_tokens >= 0.0); - // To avoid getDynamicTokens() too frequently. + // To avoid the interval between each call from being less than 1ms, + // resulting in dynamic_tokens always being 0. if (dynamic_tokens == 0.0) return; @@ -101,8 +102,8 @@ double TokenBucket::getDynamicTokens(const TokenBucket::TimePoint & timepoint) c { RUNTIME_CHECK(timepoint >= last_compact_timepoint); auto elspased = timepoint - last_compact_timepoint; - auto elapsed_second = std::chrono::duration_cast(elspased).count(); - return elapsed_second * fill_rate; + auto elapsed_ms = std::chrono::duration_cast(elspased).count(); + return elapsed_ms * fill_rate / 1000; } } // namespace DB From c7f0d53e95c848bed042869b5e15937785835cc2 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 19 Sep 2023 15:50:47 +0800 Subject: [PATCH 13/20] refine compact() Signed-off-by: guo-shaoge --- .../TaskQueues/tests/gtest_resource_control_queue.cpp | 6 +++--- dbms/src/Flash/ResourceControl/TokenBucket.cpp | 9 ++++----- dbms/src/Flash/ResourceControl/TokenBucket.h | 1 + 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index 2f2bd0a7d5c..5003117c91c 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -656,13 +656,13 @@ TEST_F(TestResourceControlQueue, tokenBucket) } { TokenBucket bucket(fill_rate, init_tokens, "log_id"); - for (int i = 0; i < 10; ++i) + for (int i = 0; i < 1000; ++i) { - std::this_thread::sleep_for(std::chrono::microseconds(500)); + std::this_thread::sleep_for(std::chrono::microseconds(5)); bucket.peek(); } ASSERT_GT(bucket.peek(), init_tokens); - ASSERT_GE(bucket.peek(), init_tokens + fill_rate * 10 * std::chrono::microseconds(500).count() / 1000000); + ASSERT_GE(bucket.peek(), init_tokens + fill_rate * 1000 * std::chrono::microseconds(5).count() / 1000000); } } diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.cpp b/dbms/src/Flash/ResourceControl/TokenBucket.cpp index 72132d58c24..c0224406b3b 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.cpp +++ b/dbms/src/Flash/ResourceControl/TokenBucket.cpp @@ -85,12 +85,11 @@ double TokenBucket::getAvgSpeedPerSec() void TokenBucket::compact(const TokenBucket::TimePoint & timepoint) { + if (timepoint - last_compact_timepoint <= MIN_COMPACT_INTERVAL) + return; + auto dynamic_tokens = getDynamicTokens(timepoint); RUNTIME_CHECK(dynamic_tokens >= 0.0); - // To avoid the interval between each call from being less than 1ms, - // resulting in dynamic_tokens always being 0. - if (dynamic_tokens == 0.0) - return; tokens += dynamic_tokens; if (tokens >= capacity) @@ -103,7 +102,7 @@ double TokenBucket::getDynamicTokens(const TokenBucket::TimePoint & timepoint) c RUNTIME_CHECK(timepoint >= last_compact_timepoint); auto elspased = timepoint - last_compact_timepoint; auto elapsed_ms = std::chrono::duration_cast(elspased).count(); - return elapsed_ms * fill_rate / 1000; + return static_cast(elapsed_ms * fill_rate) / 1000; } } // namespace DB diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.h b/dbms/src/Flash/ResourceControl/TokenBucket.h index 1fde9d1be7d..e5fbf7e28d4 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.h +++ b/dbms/src/Flash/ResourceControl/TokenBucket.h @@ -102,6 +102,7 @@ class TokenBucket final private: static constexpr auto LOW_TOKEN_THRESHOLD_RATE = 0.8; + static constexpr auto MIN_COMPACT_INTERVAL = std::chrono::milliseconds(10); // Merge dynamic token into static token. void compact(const TokenBucket::TimePoint & timepoint); From 26efb8b6fceb2084a0fd09cf358b360157f3cb7f Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 19 Sep 2023 15:57:24 +0800 Subject: [PATCH 14/20] Update dbms/src/Flash/Mpp/MinTSOScheduler.cpp Co-authored-by: SeaRise --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index af299478e2b..1330dfe793e 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -203,7 +203,7 @@ void MinTSOScheduler::releaseThreadsThenSchedule( scheduleWaitingQueries(entry, task_manager, log); if (entry.active_set.size() + entry.waiting_set.size() == 0 && entry.estimated_thread_usage == 0) { - LOG_INFO(log, "min tso scheduler_entry of resouce group {} deleted", resource_group_name); + LOG_DEBUG(log, "min tso scheduler_entry of resouce group {} deleted", resource_group_name); scheduler_entries.erase(resource_group_name); } } From 5a7f8546db31ae22356616bb47440f9c103c7175 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 19 Sep 2023 16:13:07 +0800 Subject: [PATCH 15/20] Update dbms/src/Flash/Mpp/MinTSOScheduler.cpp Co-authored-by: SeaRise --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 1330dfe793e..f112c9228af 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -201,7 +201,7 @@ void MinTSOScheduler::releaseThreadsThenSchedule( GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, resource_group_name).Decrement(); /// as tasks release some threads, so some tasks would get scheduled. scheduleWaitingQueries(entry, task_manager, log); - if (entry.active_set.size() + entry.waiting_set.size() == 0 && entry.estimated_thread_usage == 0) + if ((entry.active_set.size() + entry.waiting_set.size()) == 0 && entry.estimated_thread_usage == 0) { LOG_DEBUG(log, "min tso scheduler_entry of resouce group {} deleted", resource_group_name); scheduler_entries.erase(resource_group_name); From acff7593417ee1e0ce2de95a560cb48835713fd7 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 19 Sep 2023 16:20:42 +0800 Subject: [PATCH 16/20] RUNTIME_CHECK -> assert Signed-off-by: guo-shaoge --- dbms/src/Flash/ResourceControl/TokenBucket.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.cpp b/dbms/src/Flash/ResourceControl/TokenBucket.cpp index c0224406b3b..0c1032cdb5d 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.cpp +++ b/dbms/src/Flash/ResourceControl/TokenBucket.cpp @@ -89,8 +89,7 @@ void TokenBucket::compact(const TokenBucket::TimePoint & timepoint) return; auto dynamic_tokens = getDynamicTokens(timepoint); - RUNTIME_CHECK(dynamic_tokens >= 0.0); - + assert(dynamic_tokens >= 0.0); tokens += dynamic_tokens; if (tokens >= capacity) tokens = capacity; From c47e0a9802cc269a3afb1e9c0d6e60246524d66f Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 20 Sep 2023 11:32:27 +0800 Subject: [PATCH 17/20] scheduler_entries -> group_entries; mustGetGroupEntry Signed-off-by: guo-shaoge --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 172 ++++++++++-------- dbms/src/Flash/Mpp/MinTSOScheduler.h | 3 +- .../src/Flash/ResourceControl/TokenBucket.cpp | 4 +- 3 files changed, 101 insertions(+), 78 deletions(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index f112c9228af..a9dbb05c5e4 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -106,7 +106,7 @@ void MinTSOScheduler::deleteQuery( return; } - auto & entry = getOrCreateGroupEntry(query_id.resource_group_name); + auto & group_entry = getOrCreateGroupEntry(query_id.resource_group_name); bool all_gathers_deleted = true; auto query = task_manager.getMPPQueryWithoutLock(query_id); @@ -146,22 +146,22 @@ void MinTSOScheduler::deleteQuery( "{} query {} (is min = {}) is deleted from active set {} left {} or waiting set {} left {}.", is_cancelled ? "Cancelled" : "Finished", query_id.toString(), - query_id == entry.min_query_id, - entry.active_set.find(query_id) != entry.active_set.end(), - entry.active_set.size(), - entry.waiting_set.find(query_id) != entry.waiting_set.end(), - entry.waiting_set.size()); - entry.active_set.erase(query_id); - entry.waiting_set.erase(query_id); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) - .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, entry.resource_group_name) - .Set(entry.active_set.size()); + query_id == group_entry.min_query_id, + group_entry.active_set.find(query_id) != group_entry.active_set.end(), + group_entry.active_set.size(), + group_entry.waiting_set.find(query_id) != group_entry.waiting_set.end(), + group_entry.waiting_set.size()); + group_entry.active_set.erase(query_id); + group_entry.waiting_set.erase(query_id); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, group_entry.resource_group_name) + .Set(group_entry.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, group_entry.resource_group_name) + .Set(group_entry.active_set.size()); /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. - if (entry.updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it", log)) + if (group_entry.updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it", log)) { - scheduleWaitingQueries(entry, task_manager, log); + scheduleWaitingQueries(group_entry, task_manager, log); } } else @@ -187,44 +187,51 @@ void MinTSOScheduler::releaseThreadsThenSchedule( } global_estimated_thread_usage -= needed_threads; - auto & entry = getOrCreateGroupEntry(resource_group_name); - auto updated_estimated_threads = static_cast(entry.estimated_thread_usage) - needed_threads; + auto & group_entry = mustGetGroupEntry(resource_group_name); + auto updated_estimated_threads = static_cast(group_entry.estimated_thread_usage) - needed_threads; RUNTIME_ASSERT( updated_estimated_threads >= 0, log, "estimated_thread_usage should not be smaller than 0, actually is {}.", updated_estimated_threads); - entry.estimated_thread_usage = updated_estimated_threads; + group_entry.estimated_thread_usage = updated_estimated_threads; GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, resource_group_name) - .Set(entry.estimated_thread_usage); + .Set(group_entry.estimated_thread_usage); GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, resource_group_name).Decrement(); /// as tasks release some threads, so some tasks would get scheduled. - scheduleWaitingQueries(entry, task_manager, log); - if ((entry.active_set.size() + entry.waiting_set.size()) == 0 && entry.estimated_thread_usage == 0) + scheduleWaitingQueries(group_entry, task_manager, log); + if ((group_entry.active_set.size() + group_entry.waiting_set.size()) == 0 + && group_entry.estimated_thread_usage == 0) { LOG_DEBUG(log, "min tso scheduler_entry of resouce group {} deleted", resource_group_name); - scheduler_entries.erase(resource_group_name); + group_entries.erase(resource_group_name); } } -void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager & task_manager, LoggerPtr log) +void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & group_entry, MPPTaskManager & task_manager, LoggerPtr log) { /// schedule new tasks - while (!entry.waiting_set.empty()) + while (!group_entry.waiting_set.empty()) { - auto current_query_id = *entry.waiting_set.begin(); + auto current_query_id = *group_entry.waiting_set.begin(); auto query = task_manager.getMPPQueryWithoutLock(current_query_id); if (nullptr == query) /// silently solve this rare case { LOG_ERROR(log, "the waiting query {} is not in the task manager.", current_query_id.toString()); - entry.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); - entry.active_set.erase(current_query_id); - entry.waiting_set.erase(current_query_id); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) - .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, entry.resource_group_name) - .Set(entry.active_set.size()); + group_entry.updateMinQueryId(current_query_id, true, "as it is not in the task manager.", log); + group_entry.active_set.erase(current_query_id); + group_entry.waiting_set.erase(current_query_id); + GET_RESOURCE_GROUP_METRIC( + tiflash_task_scheduler, + type_waiting_queries_count, + group_entry.resource_group_name) + .Set(group_entry.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC( + tiflash_task_scheduler, + type_active_queries_count, + group_entry.resource_group_name) + .Set(group_entry.active_set.size()); continue; } @@ -232,8 +239,8 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager log, "query {} (is min = {}) is to be scheduled from waiting set (size = {}).", current_query_id.toString(), - current_query_id == entry.min_query_id, - entry.waiting_set.size()); + current_query_id == group_entry.min_query_id, + group_entry.waiting_set.size()); /// schedule tasks one by one for (auto & gather_set : query->mpp_gathers) { @@ -251,7 +258,7 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager GET_RESOURCE_GROUP_METRIC( tiflash_task_scheduler, type_waiting_tasks_count, - entry.resource_group_name) + group_entry.resource_group_name) .Decrement(); continue; } @@ -264,7 +271,10 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager return; } gather_set.second->waiting_tasks.pop(); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_tasks_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_task_scheduler, + type_waiting_tasks_count, + group_entry.resource_group_name) .Decrement(); } } @@ -272,11 +282,11 @@ void MinTSOScheduler::scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager log, "query {} (is min = {}) is scheduled from waiting set (size = {}).", current_query_id.toString(), - current_query_id == entry.min_query_id, - entry.waiting_set.size()); - entry.waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) - .Set(entry.waiting_set.size()); + current_query_id == group_entry.min_query_id, + group_entry.waiting_set.size()); + group_entry.waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, group_entry.resource_group_name) + .Set(group_entry.waiting_set.size()); } } @@ -289,45 +299,46 @@ bool MinTSOScheduler::scheduleImp( bool & has_error) { auto needed_threads = schedule_entry.getNeededThreads(); - auto & entry = getOrCreateGroupEntry(query_id.resource_group_name); + auto & group_entry = getOrCreateGroupEntry(query_id.resource_group_name); auto check_for_new_min_tso - = query_id <= entry.min_query_id && global_estimated_thread_usage + needed_threads <= thread_hard_limit; - auto check_for_not_min_tso - = (entry.active_set.size() < active_set_soft_limit || entry.active_set.find(query_id) != entry.active_set.end()) - && (entry.estimated_thread_usage + needed_threads <= thread_soft_limit); + = query_id <= group_entry.min_query_id && global_estimated_thread_usage + needed_threads <= thread_hard_limit; + auto check_for_not_min_tso = (group_entry.active_set.size() < active_set_soft_limit + || group_entry.active_set.find(query_id) != group_entry.active_set.end()) + && (group_entry.estimated_thread_usage + needed_threads <= thread_soft_limit); if (check_for_new_min_tso || check_for_not_min_tso) { - entry.updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it", log); - entry.active_set.insert(query_id); + group_entry + .updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it", log); + group_entry.active_set.insert(query_id); if (schedule_entry.schedule(ScheduleState::SCHEDULED)) { - entry.estimated_thread_usage += needed_threads; + group_entry.estimated_thread_usage += needed_threads; global_estimated_thread_usage += needed_threads; - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, group_entry.resource_group_name) .Increment(); } - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, entry.resource_group_name) - .Set(entry.active_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, entry.resource_group_name) - .Set(entry.estimated_thread_usage); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, group_entry.resource_group_name) + .Set(group_entry.active_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, group_entry.resource_group_name) + .Set(group_entry.estimated_thread_usage); GET_METRIC(tiflash_task_scheduler, type_global_estimated_thread_usage).Set(global_estimated_thread_usage); LOG_DEBUG( log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} " "of the thread {} limit {}.", schedule_entry.getMPPTaskId().toString(), - entry.active_set.size(), + group_entry.active_set.size(), isWaiting ? "from the waiting set" : "directly", needed_threads, - entry.estimated_thread_usage, - entry.min_query_id == query_id ? "hard" : "soft", - entry.min_query_id == query_id ? thread_hard_limit : thread_soft_limit); + group_entry.estimated_thread_usage, + group_entry.min_query_id == query_id ? "hard" : "soft", + group_entry.min_query_id == query_id ? thread_hard_limit : thread_soft_limit); return true; } else { - bool is_query_id_min = query_id <= entry.min_query_id; + bool is_query_id_min = query_id <= group_entry.min_query_id; fiu_do_on(FailPoints::random_min_tso_scheduler_failpoint, is_query_id_min = true;); if (is_query_id_min) /// the min_query_id query should fully run, otherwise throw errors here. { @@ -336,16 +347,19 @@ bool MinTSOScheduler::scheduleImp( "threads are unavailable for the query {} ({} min_query_id {}) {}, need {}, but used {} of the thread " "hard limit {}, {} active and {} waiting queries.", query_id.toString(), - query_id == entry.min_query_id ? "is" : "is newer than", - entry.min_query_id.toString(), + query_id == group_entry.min_query_id ? "is" : "is newer than", + group_entry.min_query_id.toString(), isWaiting ? "from the waiting set" : "when directly schedule it", needed_threads, - entry.estimated_thread_usage, + group_entry.estimated_thread_usage, thread_hard_limit, - entry.active_set.size(), - entry.waiting_set.size()); + group_entry.active_set.size(), + group_entry.waiting_set.size()); LOG_ERROR(log, "{}", msg); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_hard_limit_exceeded_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_task_scheduler, + type_hard_limit_exceeded_count, + group_entry.resource_group_name) .Increment(); if (isWaiting) { @@ -360,11 +374,14 @@ bool MinTSOScheduler::scheduleImp( } if (!isWaiting) { - entry.waiting_set.insert(query_id); + group_entry.waiting_set.insert(query_id); query_task_set->waiting_tasks.push(schedule_entry.getMPPTaskId()); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, entry.resource_group_name) - .Set(entry.waiting_set.size()); - GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_tasks_count, entry.resource_group_name) + GET_RESOURCE_GROUP_METRIC( + tiflash_task_scheduler, + type_waiting_queries_count, + group_entry.resource_group_name) + .Set(group_entry.waiting_set.size()); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_tasks_count, group_entry.resource_group_name) .Increment(); } LOG_INFO( @@ -374,10 +391,10 @@ bool MinTSOScheduler::scheduleImp( "required threads count are {}, waiting set size = {}", query_id.toString(), !isWaiting, - thread_soft_limit - entry.estimated_thread_usage, - active_set_soft_limit - entry.active_set.size(), + thread_soft_limit - group_entry.estimated_thread_usage, + active_set_soft_limit - group_entry.active_set.size(), needed_threads, - entry.waiting_set.size()); + group_entry.waiting_set.size()); return false; } } @@ -426,16 +443,23 @@ bool MinTSOScheduler::GroupEntry::updateMinQueryId( MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const String & resource_group_name) { - auto iter = scheduler_entries.find(resource_group_name); - if (iter == scheduler_entries.end()) + auto iter = group_entries.find(resource_group_name); + if (iter == group_entries.end()) { GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_hard_limit, resource_group_name) .Set(thread_hard_limit); GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, resource_group_name) .Set(thread_soft_limit); - iter = scheduler_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; + iter = group_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; } return iter->second; } +MinTSOScheduler::GroupEntry & MinTSOScheduler::mustGetGroupEntry(const String & resource_group_name) +{ + auto iter = group_entries.find(resource_group_name); + RUNTIME_CHECK_MSG(iter != group_entries.end(), "cannot find min tso scheduler for resource group {}", resource_group_name); + return iter->second; +} + } // namespace DB diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.h b/dbms/src/Flash/Mpp/MinTSOScheduler.h index 85ff2d177cc..3fde6e1698d 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.h +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.h @@ -80,8 +80,9 @@ class MinTSOScheduler : private boost::noncopyable void scheduleWaitingQueries(GroupEntry & entry, MPPTaskManager & task_manager, LoggerPtr log); GroupEntry & mustGetGroupEntry(const String & resource_group_name); GroupEntry & getOrCreateGroupEntry(const String & resource_group_name); + GroupEntry & mustGetGroupEntry(const String & resource_group_name) const; - std::unordered_map scheduler_entries; + std::unordered_map group_entries; UInt64 thread_soft_limit; UInt64 thread_hard_limit; UInt64 global_estimated_thread_usage; diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.cpp b/dbms/src/Flash/ResourceControl/TokenBucket.cpp index 0c1032cdb5d..ef7de26e7e1 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.cpp +++ b/dbms/src/Flash/ResourceControl/TokenBucket.cpp @@ -88,9 +88,7 @@ void TokenBucket::compact(const TokenBucket::TimePoint & timepoint) if (timepoint - last_compact_timepoint <= MIN_COMPACT_INTERVAL) return; - auto dynamic_tokens = getDynamicTokens(timepoint); - assert(dynamic_tokens >= 0.0); - tokens += dynamic_tokens; + tokens += getDynamicTokens(timepoint); if (tokens >= capacity) tokens = capacity; last_compact_timepoint = timepoint; From d4cd18b18da2c6d7f6ed0628ff81beea8c734b97 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 20 Sep 2023 11:38:38 +0800 Subject: [PATCH 18/20] add metric for group_entry count Signed-off-by: guo-shaoge --- dbms/src/Common/TiFlashMetrics.h | 3 ++- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 7 ++++++- metrics/grafana/tiflash_summary.json | 13 +++++++++++-- 3 files changed, 19 insertions(+), 4 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index e60bb886c10..8da5bcc701c 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -486,7 +486,8 @@ namespace DB F(type_estimated_thread_usage, {"type", "estimated_thread_usage"}), \ F(type_thread_soft_limit, {"type", "thread_soft_limit"}), \ F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ - F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) \ + F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"}), \ + F(type_group_entry_count, {"type", "group_entry_count"})) \ M(tiflash_task_scheduler_waiting_duration_seconds, \ "Bucketed histogram of task waiting for scheduling duration", \ Histogram, \ diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index a9dbb05c5e4..7b5023d7e4f 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -206,6 +206,7 @@ void MinTSOScheduler::releaseThreadsThenSchedule( { LOG_DEBUG(log, "min tso scheduler_entry of resouce group {} deleted", resource_group_name); group_entries.erase(resource_group_name); + GET_METRIC(tiflash_task_scheduler, type_group_entry_count).Decrement(); } } @@ -451,6 +452,7 @@ MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const Strin GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, resource_group_name) .Set(thread_soft_limit); iter = group_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; + GET_METRIC(tiflash_task_scheduler, type_group_entry_count).Increment(); } return iter->second; } @@ -458,7 +460,10 @@ MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const Strin MinTSOScheduler::GroupEntry & MinTSOScheduler::mustGetGroupEntry(const String & resource_group_name) { auto iter = group_entries.find(resource_group_name); - RUNTIME_CHECK_MSG(iter != group_entries.end(), "cannot find min tso scheduler for resource group {}", resource_group_name); + RUNTIME_CHECK_MSG( + iter != group_entries.end(), + "cannot find min tso scheduler for resource group {}", + resource_group_name); return iter->second; } diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index db32d1cb15a..c4ac1f27246 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -4333,7 +4333,16 @@ "interval": "", "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", - "refId": "C" + "refId": "D" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"group_entry_count\"}) by (instance, type)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "queryType": "randomWalk", + "refId": "E" } ], "thresholds": [], @@ -16481,4 +16490,4 @@ "title": "Test-Cluster-TiFlash-Summary", "uid": "SVbh2xUWk", "version": 1 -} \ No newline at end of file +} From bae10183ba1a82a5463c1d709cda9361c3c2b373 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 20 Sep 2023 11:53:20 +0800 Subject: [PATCH 19/20] refine metric Signed-off-by: guo-shaoge --- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 7b5023d7e4f..13dba8b4d58 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -447,11 +447,19 @@ MinTSOScheduler::GroupEntry & MinTSOScheduler::getOrCreateGroupEntry(const Strin auto iter = group_entries.find(resource_group_name); if (iter == group_entries.end()) { + iter = group_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_min_tso, resource_group_name) + .Set(iter->second.min_query_id.query_ts); GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_hard_limit, resource_group_name) .Set(thread_hard_limit); GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_thread_soft_limit, resource_group_name) .Set(thread_soft_limit); - iter = group_entries.insert({resource_group_name, GroupEntry(resource_group_name)}).first; + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_estimated_thread_usage, resource_group_name).Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_queries_count, resource_group_name).Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_queries_count, resource_group_name).Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_waiting_tasks_count, resource_group_name).Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_active_tasks_count, resource_group_name).Set(0); + GET_RESOURCE_GROUP_METRIC(tiflash_task_scheduler, type_hard_limit_exceeded_count, resource_group_name).Set(0); GET_METRIC(tiflash_task_scheduler, type_group_entry_count).Increment(); } return iter->second; From 59c0e70c1474f3069dd0f6ead7949bd1a8321c93 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 20 Sep 2023 12:51:46 +0800 Subject: [PATCH 20/20] fill_rate_ms Signed-off-by: guo-shaoge --- dbms/src/Flash/ResourceControl/TokenBucket.cpp | 3 ++- dbms/src/Flash/ResourceControl/TokenBucket.h | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.cpp b/dbms/src/Flash/ResourceControl/TokenBucket.cpp index ef7de26e7e1..6a255db8b43 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.cpp +++ b/dbms/src/Flash/ResourceControl/TokenBucket.cpp @@ -49,6 +49,7 @@ void TokenBucket::reConfig(const TokenBucketConfig & config) auto now = std::chrono::steady_clock::now(); tokens = config.tokens; fill_rate = config.fill_rate; + fill_rate_ms = config.fill_rate / 1000; capacity = config.capacity; compact(now); @@ -99,7 +100,7 @@ double TokenBucket::getDynamicTokens(const TokenBucket::TimePoint & timepoint) c RUNTIME_CHECK(timepoint >= last_compact_timepoint); auto elspased = timepoint - last_compact_timepoint; auto elapsed_ms = std::chrono::duration_cast(elspased).count(); - return static_cast(elapsed_ms * fill_rate) / 1000; + return elapsed_ms * fill_rate_ms; } } // namespace DB diff --git a/dbms/src/Flash/ResourceControl/TokenBucket.h b/dbms/src/Flash/ResourceControl/TokenBucket.h index e5fbf7e28d4..dc625f06848 100644 --- a/dbms/src/Flash/ResourceControl/TokenBucket.h +++ b/dbms/src/Flash/ResourceControl/TokenBucket.h @@ -39,6 +39,7 @@ class TokenBucket final const std::string & log_id, double capacity_ = std::numeric_limits::max()) : fill_rate(fill_rate_) + , fill_rate_ms(fill_rate_ / 1000) , tokens(init_tokens_) , capacity(capacity_) , last_compact_timepoint(std::chrono::steady_clock::now()) @@ -109,6 +110,7 @@ class TokenBucket final double getDynamicTokens(const TimePoint & timepoint) const; double fill_rate; + double fill_rate_ms; double tokens; double capacity;