ClickHouse/src/Common/OvercommitTracker.cpp

195 lines
6.7 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)
, cancellation_state(QueryCancellationState::NONE)
2022-02-16 20:02:14 +00:00
, global_mutex(global_mutex_)
, freed_memory(0)
, required_memory(0)
, allow_release(true)
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;
}
bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
2021-10-22 12:56:09 +00:00
{
// 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::onQueryStop(MemoryTracker *) is
2022-02-21 15:45:17 +00:00
// 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(cancellation_state != QueryCancellationState::NONE);
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)
{
2022-05-02 23:02:43 +00:00
// Here state can not be RUNNING, because it requires
// picked_tracker to be not null pointer.
assert(cancellation_state == QueryCancellationState::SELECTED);
cancellation_state = QueryCancellationState::NONE;
2021-10-22 12:56:09 +00:00
return true;
}
if (picked_tracker == tracker)
{
2022-05-02 23:02:43 +00:00
// Query of the provided as an argument memory tracker was chosen.
// It may happen even when current state is RUNNING, because
// ThreadStatus::~ThreadStatus may call MemoryTracker::alloc.
cancellation_state = QueryCancellationState::RUNNING;
return true;
}
allow_release = true;
required_memory += amount;
required_per_thread[tracker] = amount;
bool timeout = !cv.wait_for(lk, max_wait_time, [this, tracker]()
2021-10-22 12:56:09 +00:00
{
return required_per_thread[tracker] == 0 || cancellation_state == QueryCancellationState::NONE;
2021-10-22 12:56:09 +00:00
});
LOG_DEBUG(getLogger(), "Memory was{} freed within timeout", (timeout ? " not" : ""));
required_memory -= amount;
Int64 still_need = required_per_thread[tracker]; // If enough memory is freed it will be 0
required_per_thread.erase(tracker);
// If threads where not released since last call of this method,
// we can release them now.
if (allow_release && required_memory <= freed_memory && still_need != 0)
releaseThreads();
// All required amount of memory is free now and selected query to stop doesn't know about it.
// As we don't need to free memory, we can continue execution of the selected query.
if (required_memory == 0 && cancellation_state == QueryCancellationState::SELECTED)
reset();
return timeout || still_need != 0;
}
void OvercommitTracker::tryContinueQueryExecutionAfterFree(Int64 amount)
{
std::lock_guard guard(overcommit_m);
if (cancellation_state != QueryCancellationState::NONE)
{
freed_memory += amount;
if (freed_memory >= required_memory)
releaseThreads();
}
}
void OvercommitTracker::onQueryStop(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");
reset();
cv.notify_all();
}
}
void OvercommitTracker::releaseThreads()
{
for (auto & required : required_per_thread)
required.second = 0;
freed_memory = 0;
allow_release = false; // To avoid repeating call of this method in OvercommitTracker::needToStopQuery
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())
2022-05-04 00:41:15 +00:00
continue;
2021-12-06 22:06:00 +00:00
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)
2022-05-04 00:41:15 +00:00
continue;
2021-12-06 22:06:00 +00:00
auto * memory_tracker = query.getMemoryTracker();
if (!memory_tracker)
2022-05-04 00:41:15 +00:00
continue;
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
}