mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #17453 from azat/memory-tracking-exception-locking
Do not allow to throw MEMORY_LIMIT_EXCEEDED if there is uncaught exception
This commit is contained in:
commit
2644f446ab
@ -30,6 +30,22 @@ MemoryTracker * getMemoryTracker()
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
/// MemoryTracker cannot throw MEMORY_LIMIT_EXCEEDED (either configured memory
|
||||
/// limit reached or fault injected), in the following cases:
|
||||
///
|
||||
/// - when it is explicitly blocked with LockExceptionInThread
|
||||
///
|
||||
/// - to avoid std::terminate(), when stack unwinding is current in progress in
|
||||
/// this thread.
|
||||
///
|
||||
/// NOTE: that since C++11 destructor marked with noexcept by default, and
|
||||
/// this means that any throw from destructor (that is not marked with
|
||||
/// noexcept(false)) will cause std::terminate()
|
||||
bool inline memoryTrackerCanThrow()
|
||||
{
|
||||
return !MemoryTracker::LockExceptionInThread::isBlocked() && !std::uncaught_exceptions();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -49,6 +65,7 @@ namespace ProfileEvents
|
||||
static constexpr size_t log_peak_memory_usage_every = 1ULL << 30;
|
||||
|
||||
thread_local bool MemoryTracker::BlockerInThread::is_blocked = false;
|
||||
thread_local bool MemoryTracker::LockExceptionInThread::is_blocked = false;
|
||||
|
||||
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
|
||||
|
||||
@ -127,7 +144,7 @@ void MemoryTracker::alloc(Int64 size)
|
||||
}
|
||||
|
||||
std::bernoulli_distribution fault(fault_probability);
|
||||
if (unlikely(fault_probability && fault(thread_local_rng)))
|
||||
if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow())
|
||||
{
|
||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||
BlockerInThread untrack_lock;
|
||||
@ -156,7 +173,7 @@ void MemoryTracker::alloc(Int64 size)
|
||||
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size);
|
||||
}
|
||||
|
||||
if (unlikely(current_hard_limit && will_be > current_hard_limit))
|
||||
if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow())
|
||||
{
|
||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||
BlockerInThread untrack_lock;
|
||||
|
@ -142,6 +142,30 @@ public:
|
||||
~BlockerInThread() { is_blocked = false; }
|
||||
static bool isBlocked() { return is_blocked; }
|
||||
};
|
||||
|
||||
/// To be able to avoid MEMORY_LIMIT_EXCEEDED Exception in destructors:
|
||||
/// - either configured memory limit reached
|
||||
/// - or fault injected
|
||||
///
|
||||
/// So this will simply ignore the configured memory limit (and avoid fault injection).
|
||||
///
|
||||
/// NOTE: exception will be silently ignored, no message in log
|
||||
/// (since logging from MemoryTracker::alloc() is tricky)
|
||||
///
|
||||
/// NOTE: MEMORY_LIMIT_EXCEEDED Exception implicitly blocked if
|
||||
/// stack unwinding is currently in progress in this thread (to avoid
|
||||
/// std::terminate()), so you don't need to use it in this case explicitly.
|
||||
struct LockExceptionInThread
|
||||
{
|
||||
private:
|
||||
LockExceptionInThread(const LockExceptionInThread &) = delete;
|
||||
LockExceptionInThread & operator=(const LockExceptionInThread &) = delete;
|
||||
static thread_local bool is_blocked;
|
||||
public:
|
||||
LockExceptionInThread() { is_blocked = true; }
|
||||
~LockExceptionInThread() { is_blocked = false; }
|
||||
static bool isBlocked() { return is_blocked; }
|
||||
};
|
||||
};
|
||||
|
||||
extern MemoryTracker total_memory_tracker;
|
||||
|
@ -63,6 +63,10 @@ const Block & PullingAsyncPipelineExecutor::getHeader() const
|
||||
}
|
||||
|
||||
static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads)
|
||||
{
|
||||
setThreadName("QueryPipelineEx");
|
||||
|
||||
try
|
||||
{
|
||||
if (thread_group)
|
||||
CurrentThread::attachTo(thread_group);
|
||||
@ -72,10 +76,6 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou
|
||||
CurrentThread::detachQueryIfNotDetached();
|
||||
);
|
||||
|
||||
setThreadName("QueryPipelineEx");
|
||||
|
||||
try
|
||||
{
|
||||
data.executor->execute(num_threads);
|
||||
}
|
||||
catch (...)
|
||||
|
15
tests/queries/0_stateless/01594_too_low_memory_limits.sh
Executable file
15
tests/queries/0_stateless/01594_too_low_memory_limits.sh
Executable file
@ -0,0 +1,15 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
# it is not mandatory to use existing table since it fails earlier, hence just a placeholder.
|
||||
# this is format of INSERT SELECT, that pass these settings exactly for INSERT query not the SELECT
|
||||
${CLICKHOUSE_CLIENT} --format Null -q 'insert into placeholder_table_name select * from numbers_mt(65535) format Null settings max_memory_usage=1, max_untracked_memory=1' >& /dev/null
|
||||
exit_code=$?
|
||||
|
||||
# expecting ATTEMPT_TO_READ_AFTER_EOF, 32
|
||||
test $exit_code -eq 32 || exit 1
|
||||
|
||||
# check that server is still alive
|
||||
${CLICKHOUSE_CLIENT} --format Null -q 'SELECT 1'
|
Loading…
Reference in New Issue
Block a user