mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-18 13:42:02 +00:00
Add comments.
This commit is contained in:
parent
d27f61b4a2
commit
e451137040
@ -470,6 +470,55 @@ struct AggregateFunctionFlameGraphData
|
||||
}
|
||||
};
|
||||
|
||||
/// Aggregate function which builds a flamegraph using the list of stacktraces.
|
||||
/// The output is an array of strings which can be used by flamegraph.pl util.
|
||||
/// See https://github.com/brendangregg/FlameGraph
|
||||
///
|
||||
/// Syntax: flameGraph(traces, [size = 1], [ptr = 0])
|
||||
/// - trace : Array(UInt64), a stacktrace
|
||||
/// - size : Int64, an allocation size (for memory profiling)
|
||||
/// - ptr : UInt64, an allocation address
|
||||
/// In case if ptr != 0, a flameGraph will map allocations (size > 0) and deallocations (size < 0) with the same size and ptr.
|
||||
/// Only allocations which were not freed are shown. Not mapped deallocations are ignored.
|
||||
///
|
||||
/// Usage:
|
||||
///
|
||||
/// * Build a flamegraph based on CPU query profiler
|
||||
/// set query_profiler_cpu_time_period_ns=10000000;
|
||||
/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
|
||||
/// clickhouse client --allow_introspection_functions=1
|
||||
/// -q "select arrayJoin(flameGraph(arrayReverse(trace))) from system.trace_log where trace_type = 'CPU' and query_id = 'xxx'"
|
||||
/// | ~/dev/FlameGraph/flamegraph.pl > flame_cpu.svg
|
||||
///
|
||||
/// * Build a flamegraph based on memory query profiler, showing all allocations
|
||||
/// set memory_profiler_sample_probability=1, max_untracked_memory=1;
|
||||
/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
|
||||
/// clickhouse client --allow_introspection_functions=1
|
||||
/// -q "select arrayJoin(flameGraph(trace, size)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'"
|
||||
/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem.svg
|
||||
///
|
||||
/// * Build a flamegraph based on memory query profiler, showing allocations which were not deallocated in query context
|
||||
/// set memory_profiler_sample_probability=1, max_untracked_memory=1, use_uncompressed_cache=1, merge_tree_max_rows_to_use_cache=100000000000, merge_tree_max_bytes_to_use_cache=1000000000000;
|
||||
/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
|
||||
/// clickhouse client --allow_introspection_functions=1
|
||||
/// -q "select arrayJoin(flameGraph(trace, size, ptr)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'"
|
||||
/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_untracked.svg
|
||||
///
|
||||
/// * Build a flamegraph based on memory query profiler, showing active allocations at the fixed point of time
|
||||
/// set memory_profiler_sample_probability=1, max_untracked_memory=1;
|
||||
/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
|
||||
/// 1. Memory usage per second
|
||||
/// select event_time, m, formatReadableSize(max(s) as m) from (select event_time, sum(size) over (order by event_time) as s from system.trace_log where query_id = 'xxx' and trace_type = 'MemorySample') group by event_time order by event_time;
|
||||
/// 2. Find a time point with maximal memory usage
|
||||
/// select argMax(event_time, s), max(s) from (select event_time, sum(size) over (order by event_time) as s from system.trace_log where query_id = 'xxx' and trace_type = 'MemorySample');
|
||||
/// 3. Fix active allocations at fixed point of time
|
||||
/// clickhouse client --allow_introspection_functions=1
|
||||
/// -q "select arrayJoin(flameGraph(trace, size, ptr)) from (select * from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx' and event_time <= 'yyy' order by event_time)"
|
||||
/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_pos.svg
|
||||
/// 4. Find deallocations at fixed point of time
|
||||
/// clickhouse client --allow_introspection_functions=1
|
||||
/// -q "select arrayJoin(flameGraph(trace, -size, ptr)) from (select * from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx' and event_time > 'yyy' order by event_time desc)"
|
||||
/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_neg.svg
|
||||
class AggregateFunctionFlameGraph final : public IAggregateFunctionDataHelper<AggregateFunctionFlameGraphData, AggregateFunctionFlameGraph>
|
||||
{
|
||||
public:
|
||||
@ -489,7 +538,6 @@ public:
|
||||
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
|
||||
{
|
||||
const auto * trace = typeid_cast<const ColumnArray *>(columns[0]);
|
||||
const auto & sizes = typeid_cast<const ColumnInt64 *>(columns[1])->getData();
|
||||
|
||||
const auto & trace_offsets = trace->getOffsets();
|
||||
const auto & trace_values = typeid_cast<const ColumnUInt64 *>(&trace->getData())->getData();
|
||||
@ -498,6 +546,13 @@ public:
|
||||
prev_offset = trace_offsets[row_num - 1];
|
||||
UInt64 trace_size = trace_offsets[row_num] - prev_offset;
|
||||
|
||||
Int64 allocated = 1;
|
||||
if (argument_types.size() >= 2)
|
||||
{
|
||||
const auto & sizes = typeid_cast<const ColumnInt64 *>(columns[1])->getData();
|
||||
allocated = sizes[row_num];
|
||||
}
|
||||
|
||||
UInt64 ptr = 0;
|
||||
if (argument_types.size() >= 3)
|
||||
{
|
||||
@ -505,7 +560,7 @@ public:
|
||||
ptr = ptrs[row_num];
|
||||
}
|
||||
|
||||
this->data(place).add(ptr, sizes[row_num], trace_values.data() + prev_offset, trace_size, arena);
|
||||
this->data(place).add(ptr, allocated, trace_values.data() + prev_offset, trace_size, arena);
|
||||
}
|
||||
|
||||
void addManyDefaults(
|
||||
@ -546,10 +601,10 @@ static void check(const std::string & name, const DataTypes & argument_types, co
|
||||
{
|
||||
assertNoParameters(name, params);
|
||||
|
||||
if (argument_types.size() < 2 || argument_types.size() > 3)
|
||||
if (argument_types.empty() || argument_types.size() > 3)
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Aggregate function {} requires 2 or 3 arguments : trace, size, [ptr]",
|
||||
"Aggregate function {} requires 1 to 3 arguments : trace, [size = 1], [ptr = 0]",
|
||||
name);
|
||||
|
||||
auto ptr_type = std::make_shared<DataTypeUInt64>();
|
||||
@ -561,7 +616,7 @@ static void check(const std::string & name, const DataTypes & argument_types, co
|
||||
"First argument (trace) for function {} must be Array(UInt64), but it has type {}",
|
||||
name, argument_types[0]->getName());
|
||||
|
||||
if (!argument_types[1]->equals(*size_type))
|
||||
if (argument_types.size() >= 2 && !argument_types[1]->equals(*size_type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Second argument (size) for function {} must be Int64, but it has type {}",
|
||||
name, argument_types[1]->getName());
|
||||
|
@ -1,6 +1,9 @@
|
||||
#pragma once
|
||||
#include <cstddef>
|
||||
|
||||
/// This is a structure which is returned by MemoryTracker.
|
||||
/// Methods onAlloc/onFree should be called after actual memory allocation if it succeed.
|
||||
/// For now, it will only collect allocation trace with sample_probability.
|
||||
struct AllocationTrace
|
||||
{
|
||||
AllocationTrace() = default;
|
||||
|
@ -37,11 +37,14 @@ void CachedCompressedReadBuffer::prefetch()
|
||||
|
||||
bool CachedCompressedReadBuffer::nextImpl()
|
||||
{
|
||||
std::cerr << "============== CachedCompressedReadBuffer::nextImpl()\n";
|
||||
/// Let's check for the presence of a decompressed block in the cache, grab the ownership of this block, if it exists.
|
||||
UInt128 key = cache->hash(path, file_pos);
|
||||
|
||||
owned_cell = cache->getOrSet(key, [&]()
|
||||
{
|
||||
|
||||
std::cerr << "=========== CachedCompressedReadBuffer::getOrSet()\n";
|
||||
initInput();
|
||||
file_in->seek(file_pos, SEEK_SET);
|
||||
|
||||
|
@ -88,7 +88,7 @@ MergeListElement::MergeListElement(
|
||||
/// thread_group::memory_tracker, but MemoryTrackerThreadSwitcher will reset parent).
|
||||
memory_tracker.setProfilerStep(settings.memory_profiler_step);
|
||||
memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
|
||||
/// Specify sample probability also for current thread ot track more deallocations.
|
||||
/// Specify sample probability also for current thread to track more deallocations.
|
||||
if (auto * thread_memory_tracker = DB::CurrentThread::getMemoryTracker())
|
||||
thread_memory_tracker->setSampleProbability(settings.memory_profiler_sample_probability);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user