ClickHouse/src/Common/OvercommitTracker.cpp

148 lines
4.9 KiB
C++
Raw Normal View History

2021-10-22 12:56:09 +00:00
#include "OvercommitTracker.h"
2021-10-22 15:15:33 +00:00
#include <chrono>
2021-12-06 22:16:09 +00:00
#include <mutex>
2021-10-22 12:56:09 +00:00
#include <Interpreters/ProcessList.h>
2021-10-22 15:15:33 +00:00
using namespace std::chrono_literals;
constexpr std::chrono::microseconds ZERO_MICROSEC = 0us;
2022-02-16 20:02:14 +00:00
OvercommitTracker::OvercommitTracker(std::mutex & global_mutex_)
: max_wait_time(ZERO_MICROSEC)
2021-10-22 15:15:33 +00:00
, picked_tracker(nullptr)
, cancelation_state(QueryCancelationState::NONE)
2022-02-16 20:02:14 +00:00
, global_mutex(global_mutex_)
2021-10-22 15:15:33 +00:00
{}
void OvercommitTracker::setMaxWaitTime(UInt64 wait_time)
{
2022-01-18 12:21:59 +00:00
std::lock_guard guard(overcommit_m);
2021-10-22 15:15:33 +00:00
max_wait_time = wait_time * 1us;
}
2021-10-22 12:56:09 +00:00
bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker)
{
// NOTE: Do not change the order of locks
2022-02-21 15:45:17 +00:00
//
// global_mutex must be acquired before overcommit_m, because
// method OvercommitTracker::unsubscribe(MemoryTracker *) is
// always called with already acquired global_mutex in
// ProcessListEntry::~ProcessListEntry().
2022-02-16 20:02:14 +00:00
std::unique_lock<std::mutex> global_lock(global_mutex);
2021-10-22 12:56:09 +00:00
std::unique_lock<std::mutex> lk(overcommit_m);
if (max_wait_time == ZERO_MICROSEC)
return true;
2021-10-22 12:56:09 +00:00
pickQueryToExclude();
assert(cancelation_state == QueryCancelationState::RUNNING);
2022-02-16 20:02:14 +00:00
global_lock.unlock();
2022-01-18 12:21:59 +00:00
// If no query was chosen we need to stop current query.
// This may happen if no soft limit is set.
if (picked_tracker == nullptr)
{
cancelation_state = QueryCancelationState::NONE;
2021-10-22 12:56:09 +00:00
return true;
}
if (picked_tracker == tracker)
return true;
2022-02-16 20:02:14 +00:00
bool timeout = !cv.wait_for(lk, max_wait_time, [this]()
2021-10-22 12:56:09 +00:00
{
return cancelation_state == QueryCancelationState::NONE;
});
2022-02-16 20:02:14 +00:00
if (timeout)
LOG_DEBUG(getLogger(), "Need to stop query because reached waiting timeout");
else
LOG_DEBUG(getLogger(), "Memory freed within timeout");
return timeout;
2021-10-22 12:56:09 +00:00
}
void OvercommitTracker::unsubscribe(MemoryTracker * tracker)
{
std::unique_lock<std::mutex> lk(overcommit_m);
if (picked_tracker == tracker)
{
2022-01-18 12:21:59 +00:00
LOG_DEBUG(getLogger(), "Picked query stopped");
picked_tracker = nullptr;
cancelation_state = QueryCancelationState::NONE;
cv.notify_all();
}
}
2022-02-16 20:02:14 +00:00
UserOvercommitTracker::UserOvercommitTracker(DB::ProcessList * process_list, DB::ProcessListForUser * user_process_list_)
: OvercommitTracker(process_list->mutex)
, user_process_list(user_process_list_)
2021-10-22 12:56:09 +00:00
{}
void UserOvercommitTracker::pickQueryToExcludeImpl()
{
2022-01-18 12:21:59 +00:00
MemoryTracker * query_tracker = nullptr;
2021-10-22 12:56:09 +00:00
OvercommitRatio current_ratio{0, 0};
2022-01-18 12:21:59 +00:00
// At this moment query list must be read only.
2022-02-21 15:45:17 +00:00
// This is guaranteed by locking global_mutex in OvercommitTracker::needToStopQuery.
auto & queries = user_process_list->queries;
2021-12-06 22:06:00 +00:00
LOG_DEBUG(logger, "Trying to choose query to stop from {} queries", queries.size());
for (auto const & query : queries)
2021-10-22 12:56:09 +00:00
{
if (query.second->isKilled())
continue;
2021-10-22 12:56:09 +00:00
auto * memory_tracker = query.second->getMemoryTracker();
if (!memory_tracker)
continue;
2021-10-22 12:56:09 +00:00
auto ratio = memory_tracker->getOvercommitRatio();
2021-12-06 22:06:00 +00:00
LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit);
if (ratio.soft_limit != 0 && current_ratio < ratio)
2021-10-22 12:56:09 +00:00
{
2022-01-18 12:21:59 +00:00
query_tracker = memory_tracker;
2021-10-22 12:56:09 +00:00
current_ratio = ratio;
}
}
2021-12-06 22:06:00 +00:00
LOG_DEBUG(logger, "Selected to stop query with overcommit ratio {}/{}",
current_ratio.committed, current_ratio.soft_limit);
2022-01-18 12:21:59 +00:00
picked_tracker = query_tracker;
2021-10-22 12:56:09 +00:00
}
2022-02-16 20:02:14 +00:00
GlobalOvercommitTracker::GlobalOvercommitTracker(DB::ProcessList * process_list_)
: OvercommitTracker(process_list_->mutex)
, process_list(process_list_)
{}
2021-10-22 12:56:09 +00:00
void GlobalOvercommitTracker::pickQueryToExcludeImpl()
{
2022-01-18 12:21:59 +00:00
MemoryTracker * query_tracker = nullptr;
2021-10-22 12:56:09 +00:00
OvercommitRatio current_ratio{0, 0};
2022-02-15 15:04:13 +00:00
// At this moment query list must be read only.
2022-02-21 15:45:17 +00:00
// This is guaranteed by locking global_mutex in OvercommitTracker::needToStopQuery.
LOG_DEBUG(logger, "Trying to choose query to stop from {} queries", process_list->size());
for (auto const & query : process_list->processes)
2021-10-22 12:56:09 +00:00
{
2021-12-06 22:06:00 +00:00
if (query.isKilled())
return;
Int64 user_soft_limit = 0;
if (auto const * user_process_list = query.getUserProcessList())
user_soft_limit = user_process_list->user_memory_tracker.getSoftLimit();
2021-12-06 22:06:00 +00:00
if (user_soft_limit == 0)
return;
2021-12-06 22:06:00 +00:00
auto * memory_tracker = query.getMemoryTracker();
if (!memory_tracker)
return;
auto ratio = memory_tracker->getOvercommitRatio(user_soft_limit);
2021-12-06 22:06:00 +00:00
LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit);
2021-10-22 12:56:09 +00:00
if (current_ratio < ratio)
{
2022-01-18 12:21:59 +00:00
query_tracker = memory_tracker;
2021-10-22 12:56:09 +00:00
current_ratio = ratio;
}
2022-02-21 15:45:17 +00:00
}
2021-12-06 22:06:00 +00:00
LOG_DEBUG(logger, "Selected to stop query with overcommit ratio {}/{}",
current_ratio.committed, current_ratio.soft_limit);
2022-01-18 12:21:59 +00:00
picked_tracker = query_tracker;
2021-10-22 12:56:09 +00:00
}