Merge pull request #52496 from ClickHouse/revert-52419-implement_borders_for_sampling_memory_profiler

Revert "Add an ability to specify allocations size for sampling memory profiler"
This commit is contained in:
Alexey Milovidov 2023-07-23 19:01:03 +03:00 committed by GitHub
commit 2b25a81925
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 11 additions and 124 deletions

View File

@ -1637,26 +1637,17 @@ try
global_context->initializeTraceCollector();
/// Set up server-wide memory profiler (for total memory tracker).
if (server_settings.total_memory_profiler_step)
UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0);
if (total_memory_profiler_step)
{
total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step);
total_memory_tracker.setProfilerStep(total_memory_profiler_step);
}
if (server_settings.total_memory_tracker_sample_probability > 0.0)
double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0);
if (total_memory_tracker_sample_probability > 0.0)
{
total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability);
total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability);
}
if (server_settings.total_memory_profiler_sample_min_allocation_size)
{
total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size);
}
if (server_settings.total_memory_profiler_sample_max_allocation_size)
{
total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size);
}
}
#endif

View File

@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
}
std::bernoulli_distribution sample(sample_probability);
if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng)))
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
{
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size});
@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size)
}
std::bernoulli_distribution sample(sample_probability);
if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng)))
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
{
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size});
@ -534,12 +534,6 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value)
;
}
bool MemoryTracker::isSizeOkForSampling(UInt64 size) const
{
/// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation
return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes);
}
bool canEnqueueBackgroundTask()
{
auto limit = background_memory_tracker.getSoftLimit();

View File

@ -67,12 +67,6 @@ private:
/// To randomly sample allocations and deallocations in trace_log.
double sample_probability = 0;
/// Randomly sample allocations only larger or equal to this size
UInt64 min_allocation_size_bytes = 0;
/// Randomly sample allocations only smaller or equal to this size
UInt64 max_allocation_size_bytes = 0;
/// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy).
/// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker.
std::atomic<MemoryTracker *> parent {};
@ -94,8 +88,6 @@ private:
void setOrRaiseProfilerLimit(Int64 value);
bool isSizeOkForSampling(UInt64 size) const;
/// allocImpl(...) and free(...) should not be used directly
friend struct CurrentMemoryTracker;
void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr);
@ -173,16 +165,6 @@ public:
sample_probability = value;
}
void setSampleMinAllocationSize(UInt64 value)
{
min_allocation_size_bytes = value;
}
void setSampleMaxAllocationSize(UInt64 value)
{
max_allocation_size_bytes = value;
}
void setProfilerStep(Int64 value)
{
profiler_step = value;

View File

@ -81,12 +81,8 @@ namespace DB
M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \
\
M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \
M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0)
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0)
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)

View File

@ -427,9 +427,7 @@ class IColumn;
M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \
\
M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \

View File

@ -223,10 +223,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
{
/// Set up memory profiling
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size);
thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size);
thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events);
}

View File

@ -83,8 +83,6 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex
const Settings & settings = storage_context->getSettingsRef();
group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size);
group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size);
group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator);
group->memory_tracker.setParent(&background_memory_tracker);
if (settings.memory_tracker_fault_probability > 0.0)

View File

@ -1 +0,0 @@
#!/usr/bin/env python3

View File

@ -1,7 +0,0 @@
<clickhouse>
<profiles>
<default>
<max_untracked_memory>1</max_untracked_memory>
</default>
</profiles>
</clickhouse>

View File

@ -1,5 +0,0 @@
<clickhouse>
<total_memory_tracker_sample_probability>1</total_memory_tracker_sample_probability>
<total_memory_profiler_sample_min_allocation_size>4096</total_memory_profiler_sample_min_allocation_size>
<total_memory_profiler_sample_max_allocation_size>8192</total_memory_profiler_sample_max_allocation_size>
</clickhouse>

View File

@ -1,37 +0,0 @@
from helpers.cluster import ClickHouseCluster
import pytest
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=["configs/memory_profiler.xml"],
user_configs=["configs/max_untracked_memory.xml"],
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_trace_boundaries_work(started_cluster):
node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null")
node.query("SYSTEM FLUSH LOGS")
assert (
node.query(
"SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)"
)
== "0\n"
)

View File

@ -1,18 +0,0 @@
#!/usr/bin/env bash
# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings
# requires TraceCollector, does not available under sanitizers and aarch64
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM"
${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null"
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
# at least something allocated
${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'"
# show wrong allocations
${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)"