mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Overcommit: update defaults, exception message and add ProfileEvent
This commit is contained in:
parent
c0d5705891
commit
e9187ec4b7
@ -1095,7 +1095,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
|
||||
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
|
||||
UInt64 max_overcommit_wait_time = config->getUInt64("global_memory_usage_overcommit_max_wait_microseconds", 200);
|
||||
UInt64 max_overcommit_wait_time = config->getUInt64("global_memory_usage_overcommit_max_wait_microseconds", 5'000'000);
|
||||
global_overcommit_tracker->setMaxWaitTime(max_overcommit_wait_time);
|
||||
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);
|
||||
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <cmath>
|
||||
#include <random>
|
||||
#include <cstdlib>
|
||||
#include <string>
|
||||
|
||||
|
||||
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
|
||||
@ -52,6 +53,30 @@ namespace DB
|
||||
}
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
inline std::string_view toDescription(OvercommitResult result)
|
||||
{
|
||||
switch (result)
|
||||
{
|
||||
case OvercommitResult::NONE:
|
||||
return "Memory overcommit isn't used. OvercommitTracker isn't set.";
|
||||
case OvercommitResult::DISABLED:
|
||||
return "Memory overcommit isn't used. Waiting time or orvercommit denominator are set to zero.";
|
||||
case OvercommitResult::MEMORY_FREED:
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "OvercommitResult::MEMORY_FREED shouldn't be asked for description");
|
||||
case OvercommitResult::SELECTED:
|
||||
return "Query was selected to stop by OvercommitTracker.";
|
||||
case OvercommitResult::TIMEOUTED:
|
||||
return "Waiting timeout for memory to be freed is reached.";
|
||||
case OvercommitResult::NOT_ENOUGH_FREED:
|
||||
return "Memory overcommit has freed not enough memory.";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event QueryMemoryLimitExceeded;
|
||||
@ -189,11 +214,11 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
|
||||
|
||||
if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded)
|
||||
{
|
||||
bool need_to_throw = true;
|
||||
OvercommitResult overcommit_result = OvercommitResult::NONE;
|
||||
if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed); overcommit_tracker_ptr != nullptr && query_tracker != nullptr)
|
||||
need_to_throw = overcommit_tracker_ptr->needToStopQuery(query_tracker, size);
|
||||
overcommit_result = overcommit_tracker_ptr->needToStopQuery(query_tracker, size);
|
||||
|
||||
if (need_to_throw)
|
||||
if (overcommit_result != OvercommitResult::MEMORY_FREED)
|
||||
{
|
||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
||||
@ -201,12 +226,13 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
|
||||
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
||||
throw DB::Exception(
|
||||
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
||||
"Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
||||
"Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}. OvercommitTracker decision: {}.",
|
||||
description ? " " : "",
|
||||
description ? description : "",
|
||||
formatReadableSizeWithBinarySuffix(will_be),
|
||||
size,
|
||||
formatReadableSizeWithBinarySuffix(current_hard_limit));
|
||||
formatReadableSizeWithBinarySuffix(current_hard_limit),
|
||||
toDescription(overcommit_result));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -2,8 +2,14 @@
|
||||
|
||||
#include <chrono>
|
||||
#include <mutex>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MemoryOvercommitWaitTimeMicroseconds;
|
||||
}
|
||||
|
||||
using namespace std::chrono_literals;
|
||||
|
||||
constexpr std::chrono::microseconds ZERO_MICROSEC = 0us;
|
||||
@ -24,7 +30,7 @@ void OvercommitTracker::setMaxWaitTime(UInt64 wait_time)
|
||||
max_wait_time = wait_time * 1us;
|
||||
}
|
||||
|
||||
bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
|
||||
OvercommitResult OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
|
||||
{
|
||||
// NOTE: Do not change the order of locks
|
||||
//
|
||||
@ -36,7 +42,7 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
|
||||
std::unique_lock<std::mutex> lk(overcommit_m);
|
||||
|
||||
if (max_wait_time == ZERO_MICROSEC)
|
||||
return true;
|
||||
return OvercommitResult::DISABLED;
|
||||
|
||||
pickQueryToExclude();
|
||||
assert(cancellation_state != QueryCancellationState::NONE);
|
||||
@ -50,7 +56,7 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
|
||||
// picked_tracker to be not null pointer.
|
||||
assert(cancellation_state == QueryCancellationState::SELECTED);
|
||||
cancellation_state = QueryCancellationState::NONE;
|
||||
return true;
|
||||
return OvercommitResult::DISABLED;
|
||||
}
|
||||
if (picked_tracker == tracker)
|
||||
{
|
||||
@ -58,17 +64,20 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
|
||||
// It may happen even when current state is RUNNING, because
|
||||
// ThreadStatus::~ThreadStatus may call MemoryTracker::alloc.
|
||||
cancellation_state = QueryCancellationState::RUNNING;
|
||||
return true;
|
||||
return OvercommitResult::SELECTED;
|
||||
}
|
||||
|
||||
allow_release = true;
|
||||
|
||||
required_memory += amount;
|
||||
required_per_thread[tracker] = amount;
|
||||
auto wait_start_time = std::chrono::system_clock::now();
|
||||
bool timeout = !cv.wait_for(lk, max_wait_time, [this, tracker]()
|
||||
{
|
||||
return required_per_thread[tracker] == 0 || cancellation_state == QueryCancellationState::NONE;
|
||||
});
|
||||
auto wait_end_time = std::chrono::system_clock::now();
|
||||
ProfileEvents::increment(ProfileEvents::MemoryOvercommitWaitTimeMicroseconds, (wait_end_time - wait_start_time) / 1us);
|
||||
LOG_DEBUG(getLogger(), "Memory was{} freed within timeout", (timeout ? " not" : ""));
|
||||
|
||||
required_memory -= amount;
|
||||
@ -84,7 +93,12 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
|
||||
// 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;
|
||||
if (timeout)
|
||||
return OvercommitResult::TIMEOUTED;
|
||||
if (still_need != 0)
|
||||
return OvercommitResult::NOT_ENOUGH_FREED;
|
||||
else
|
||||
return OvercommitResult::MEMORY_FREED;
|
||||
}
|
||||
|
||||
void OvercommitTracker::tryContinueQueryExecutionAfterFree(Int64 amount)
|
||||
|
@ -36,6 +36,16 @@ struct OvercommitRatio
|
||||
|
||||
class MemoryTracker;
|
||||
|
||||
enum class OvercommitResult
|
||||
{
|
||||
NONE,
|
||||
DISABLED,
|
||||
MEMORY_FREED,
|
||||
SELECTED,
|
||||
TIMEOUTED,
|
||||
NOT_ENOUGH_FREED,
|
||||
};
|
||||
|
||||
enum class QueryCancellationState
|
||||
{
|
||||
NONE = 0, // Hard limit is not reached, there is no selected query to kill.
|
||||
@ -54,7 +64,7 @@ struct OvercommitTracker : boost::noncopyable
|
||||
{
|
||||
void setMaxWaitTime(UInt64 wait_time);
|
||||
|
||||
bool needToStopQuery(MemoryTracker * tracker, Int64 amount);
|
||||
OvercommitResult needToStopQuery(MemoryTracker * tracker, Int64 amount);
|
||||
|
||||
void tryContinueQueryExecutionAfterFree(Int64 amount);
|
||||
|
||||
|
@ -192,6 +192,7 @@
|
||||
M(RealTimeMicroseconds, "Total (wall clock) time spent in processing (queries and other tasks) threads (not that this is a sum).") \
|
||||
M(UserTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in user space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
|
||||
M(SystemTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in OS kernel space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
|
||||
M(MemoryOvercommitWaitTimeMicroseconds, "Total time spent in waiting for memory to be freed in OvercommitTracker.") \
|
||||
M(SoftPageFaults, "") \
|
||||
M(HardPageFaults, "") \
|
||||
\
|
||||
|
@ -56,7 +56,7 @@ void free_not_continue_test(T & overcommit_tracker)
|
||||
threads.push_back(std::thread(
|
||||
[&, i]()
|
||||
{
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
|
||||
++need_to_stop;
|
||||
}
|
||||
));
|
||||
@ -112,7 +112,7 @@ void free_continue_test(T & overcommit_tracker)
|
||||
threads.push_back(std::thread(
|
||||
[&, i]()
|
||||
{
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
|
||||
++need_to_stop;
|
||||
}
|
||||
));
|
||||
@ -168,7 +168,7 @@ void free_continue_and_alloc_test(T & overcommit_tracker)
|
||||
threads.push_back(std::thread(
|
||||
[&, i]()
|
||||
{
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
|
||||
++need_to_stop;
|
||||
}
|
||||
));
|
||||
@ -181,7 +181,7 @@ void free_continue_and_alloc_test(T & overcommit_tracker)
|
||||
MemoryTracker failed;
|
||||
std::this_thread::sleep_for(1000ms);
|
||||
overcommit_tracker.tryContinueQueryExecutionAfterFree(5000);
|
||||
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100);
|
||||
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED;
|
||||
}
|
||||
).join();
|
||||
|
||||
@ -228,7 +228,7 @@ void free_continue_and_alloc_2_test(T & overcommit_tracker)
|
||||
threads.push_back(std::thread(
|
||||
[&, i]()
|
||||
{
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
|
||||
++need_to_stop;
|
||||
}
|
||||
));
|
||||
@ -241,7 +241,7 @@ void free_continue_and_alloc_2_test(T & overcommit_tracker)
|
||||
MemoryTracker failed;
|
||||
std::this_thread::sleep_for(1000ms);
|
||||
overcommit_tracker.tryContinueQueryExecutionAfterFree(5000);
|
||||
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100);
|
||||
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED;
|
||||
}
|
||||
));
|
||||
|
||||
@ -296,7 +296,7 @@ void free_continue_and_alloc_3_test(T & overcommit_tracker)
|
||||
threads.push_back(std::thread(
|
||||
[&, i]()
|
||||
{
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
|
||||
++need_to_stop;
|
||||
}
|
||||
));
|
||||
@ -309,7 +309,7 @@ void free_continue_and_alloc_3_test(T & overcommit_tracker)
|
||||
MemoryTracker failed;
|
||||
std::this_thread::sleep_for(1000ms);
|
||||
overcommit_tracker.tryContinueQueryExecutionAfterFree(5000);
|
||||
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100);
|
||||
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED;
|
||||
}
|
||||
));
|
||||
|
||||
@ -364,7 +364,7 @@ void free_continue_2_test(T & overcommit_tracker)
|
||||
threads.push_back(std::thread(
|
||||
[&, i]()
|
||||
{
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
|
||||
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
|
||||
++need_to_stop;
|
||||
}
|
||||
));
|
||||
@ -415,7 +415,7 @@ void query_stop_not_continue_test(T & overcommit_tracker)
|
||||
auto thread = std::thread(
|
||||
[&]()
|
||||
{
|
||||
if (overcommit_tracker.needToStopQuery(&another, 100))
|
||||
if (overcommit_tracker.needToStopQuery(&another, 100) != OvercommitResult::MEMORY_FREED)
|
||||
++need_to_stop;
|
||||
}
|
||||
);
|
||||
|
Loading…
Reference in New Issue
Block a user