Fix wrong attribution of untracked memory to a user/query (#56089)

* impl

* add tests

* add tests

* fix typo
This commit is contained in:
Nikita Taranov 2023-11-02 17:14:15 +01:00 committed by GitHub
parent 76e6b75ae2
commit 3069db71f1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 64 additions and 10 deletions

View File

@ -1,18 +1,19 @@
#include "MemoryTracker.h"
#include <IO/WriteHelpers.h>
#include <Common/HashTable/Hash.h>
#include <Common/VariableContext.h>
#include <Common/TraceSender.h>
#include <Common/Exception.h>
#include <Common/HashTable/Hash.h>
#include <Common/LockMemoryExceptionInThread.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/formatReadable.h>
#include <Common/ProfileEvents.h>
#include <Common/thread_local_rng.h>
#include <Common/OvercommitTracker.h>
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadStatus.h>
#include <Common/TraceSender.h>
#include <Common/VariableContext.h>
#include <Common/formatReadable.h>
#include <Common/logger_useful.h>
#include <Common/thread_local_rng.h>
#include "config.h"
@ -589,6 +590,16 @@ bool MemoryTracker::isSizeOkForSampling(UInt64 size) const
return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes);
}
void MemoryTracker::setParent(MemoryTracker * elem)
{
/// Untracked memory shouldn't be accounted to a query or a user if it was allocated before the thread was attached
/// to a query thread group or a user group, because this memory will be (🤞) freed outside of these scopes.
if (level == VariableContext::Thread && DB::current_thread)
DB::current_thread->flushUntrackedMemory();
parent.store(elem, std::memory_order_relaxed);
}
bool canEnqueueBackgroundTask()
{
auto limit = background_memory_tracker.getSoftLimit();

View File

@ -200,10 +200,7 @@ public:
/// next should be changed only once: from nullptr to some value.
/// NOTE: It is not true in MergeListElement
void setParent(MemoryTracker * elem)
{
parent.store(elem, std::memory_order_relaxed);
}
void setParent(MemoryTracker * elem);
MemoryTracker * getParent()
{

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
# Tags: no-parallel, long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
total_iterations=16
parallelism=32
$CLICKHOUSE_CLIENT --query='DROP TABLE IF EXISTS test_inserts'
$CLICKHOUSE_CLIENT --query='CREATE TABLE test_inserts ENGINE=Null AS system.numbers'
run_query() {
( $CLICKHOUSE_CLIENT --query='SELECT * FROM numbers_mt(1000000) FORMAT CSV' | $CLICKHOUSE_CLIENT --max_threads 8 --max_memory_usage_for_user 1073741824 -q 'INSERT INTO test_inserts FORMAT CSV' 2>/dev/null )
}
for ((i = 1; i <= total_iterations; i++)); do
for ((j = 1; j <= parallelism; j++)); do
run_query & pids+=($!)
done
EXIT_CODE=0
new_pids=()
for pid in "${pids[@]:0:parallelism}"; do
CODE=0
wait "${pid}" || CODE=$?
run_query & new_pids+=($!)
if [[ "${CODE}" != "0" ]]; then
EXIT_CODE=1;
fi
done
for pid in "${pids[@]:parallelism}"; do
CODE=0
wait "${pid}" || CODE=$?
if [[ "${CODE}" != "0" ]]; then
EXIT_CODE=1;
fi
done
pids=("${new_pids[@]}")
if [[ $EXIT_CODE -ne 0 ]]; then
exit $EXIT_CODE
fi
done