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:
alexey-milovidov 2020-12-01 10:25:17 +03:00 committed by GitHub
commit 2644f446ab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 66 additions and 10 deletions

View File

@ -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;

View File

@ -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;

View File

@ -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 (...)

View 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'