mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #10598 from ClickHouse/sampling-memory-profiler
Add sampling memory profiler
This commit is contained in:
commit
0cdd48e468
@ -630,6 +630,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
total_memory_tracker.setOrRaiseProfilerLimit(total_memory_profiler_step);
|
||||
total_memory_tracker.setProfilerStep(total_memory_profiler_step);
|
||||
}
|
||||
|
||||
double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0);
|
||||
if (total_memory_tracker_sample_probability)
|
||||
{
|
||||
total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability);
|
||||
}
|
||||
}
|
||||
#endif
|
||||
|
||||
|
@ -139,10 +139,18 @@
|
||||
|
||||
<!-- Simple server-wide memory profiler. Collect a stack trace at every peak allocation step (in bytes).
|
||||
Data will be stored in system.trace_log table with query_id = empty string.
|
||||
Zero means disabled. Minimal effective value is 4 MiB.
|
||||
Zero means disabled.
|
||||
-->
|
||||
<total_memory_profiler_step>4194304</total_memory_profiler_step>
|
||||
|
||||
<!-- 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 the untracked memory limit,
|
||||
which is 4 MiB by default but can be lowered if 'total_memory_profiler_step' is lowered.
|
||||
You may want to set 'total_memory_profiler_step' to 1 for extra fine grained sampling.
|
||||
-->
|
||||
<total_memory_tracker_sample_probability>0</total_memory_tracker_sample_probability>
|
||||
|
||||
<!-- Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve
|
||||
correct maximum value. -->
|
||||
<!-- <max_open_files>262144</max_open_files> -->
|
||||
|
@ -52,12 +52,6 @@ public:
|
||||
static ProfileEvents::Counters & getProfileEvents();
|
||||
static MemoryTracker * getMemoryTracker();
|
||||
|
||||
static inline Int64 & getUntrackedMemory()
|
||||
{
|
||||
/// It assumes that (current_thread != nullptr) is already checked with getMemoryTracker()
|
||||
return current_thread->untracked_memory;
|
||||
}
|
||||
|
||||
/// Update read and write rows (bytes) statistics (used in system.query_thread_log)
|
||||
static void updateProgressIn(const Progress & value);
|
||||
static void updateProgressOut(const Progress & value);
|
||||
|
@ -9,6 +9,7 @@
|
||||
|
||||
#include <atomic>
|
||||
#include <cmath>
|
||||
#include <random>
|
||||
#include <cstdlib>
|
||||
|
||||
|
||||
@ -22,8 +23,6 @@ namespace DB
|
||||
|
||||
|
||||
static constexpr size_t log_peak_memory_usage_every = 1ULL << 30;
|
||||
/// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters.
|
||||
static constexpr Int64 untracked_memory_limit = 4 * 1024 * 1024;
|
||||
|
||||
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
|
||||
|
||||
@ -80,9 +79,8 @@ void MemoryTracker::alloc(Int64 size)
|
||||
Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed);
|
||||
Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed);
|
||||
|
||||
/// Using non-thread-safe random number generator. Joint distribution in different threads would not be uniform.
|
||||
/// In this case, it doesn't matter.
|
||||
if (unlikely(fault_probability && drand48() < fault_probability))
|
||||
std::bernoulli_distribution fault(fault_probability);
|
||||
if (unlikely(fault_probability && fault(thread_local_rng)))
|
||||
{
|
||||
free(size);
|
||||
|
||||
@ -107,12 +105,19 @@ void MemoryTracker::alloc(Int64 size)
|
||||
setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step);
|
||||
}
|
||||
|
||||
std::bernoulli_distribution sample(sample_probability);
|
||||
if (unlikely(sample_probability && sample(thread_local_rng)))
|
||||
{
|
||||
auto no_track = blocker.cancel();
|
||||
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size);
|
||||
}
|
||||
|
||||
if (unlikely(current_hard_limit && will_be > current_hard_limit))
|
||||
{
|
||||
free(size);
|
||||
|
||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||
auto untrack_lock = blocker.cancel(); // NOLINT
|
||||
auto no_track = blocker.cancel(); // NOLINT
|
||||
|
||||
std::stringstream message;
|
||||
message << "Memory limit";
|
||||
@ -151,6 +156,13 @@ void MemoryTracker::free(Int64 size)
|
||||
if (blocker.isCancelled())
|
||||
return;
|
||||
|
||||
std::bernoulli_distribution sample(sample_probability);
|
||||
if (unlikely(sample_probability && sample(thread_local_rng)))
|
||||
{
|
||||
auto no_track = blocker.cancel();
|
||||
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), -size);
|
||||
}
|
||||
|
||||
if (level == VariableContext::Thread)
|
||||
{
|
||||
/// Could become negative if memory allocated in this thread is freed in another one
|
||||
@ -225,18 +237,19 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value)
|
||||
|
||||
namespace CurrentMemoryTracker
|
||||
{
|
||||
using DB::current_thread;
|
||||
|
||||
void alloc(Int64 size)
|
||||
{
|
||||
if (auto * memory_tracker = DB::CurrentThread::getMemoryTracker())
|
||||
{
|
||||
Int64 & untracked = DB::CurrentThread::getUntrackedMemory();
|
||||
untracked += size;
|
||||
if (untracked > untracked_memory_limit)
|
||||
current_thread->untracked_memory += size;
|
||||
if (current_thread->untracked_memory > current_thread->untracked_memory_limit)
|
||||
{
|
||||
/// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes
|
||||
/// more. It could be useful to enlarge Exception message in rethrow logic.
|
||||
Int64 tmp = untracked;
|
||||
untracked = 0;
|
||||
Int64 tmp = current_thread->untracked_memory;
|
||||
current_thread->untracked_memory = 0;
|
||||
memory_tracker->alloc(tmp);
|
||||
}
|
||||
}
|
||||
@ -252,12 +265,11 @@ namespace CurrentMemoryTracker
|
||||
{
|
||||
if (auto * memory_tracker = DB::CurrentThread::getMemoryTracker())
|
||||
{
|
||||
Int64 & untracked = DB::CurrentThread::getUntrackedMemory();
|
||||
untracked -= size;
|
||||
if (untracked < -untracked_memory_limit)
|
||||
current_thread->untracked_memory -= size;
|
||||
if (current_thread->untracked_memory < -current_thread->untracked_memory_limit)
|
||||
{
|
||||
memory_tracker->free(-untracked);
|
||||
untracked = 0;
|
||||
memory_tracker->free(-current_thread->untracked_memory);
|
||||
current_thread->untracked_memory = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -24,6 +24,9 @@ private:
|
||||
/// To test exception safety of calling code, memory tracker throws an exception on each memory allocation with specified probability.
|
||||
double fault_probability = 0;
|
||||
|
||||
/// To randomly sample allocations and deallocations in trace_log.
|
||||
double sample_probability = 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 {};
|
||||
@ -83,6 +86,11 @@ public:
|
||||
fault_probability = value;
|
||||
}
|
||||
|
||||
void setSampleProbability(double value)
|
||||
{
|
||||
sample_probability = value;
|
||||
}
|
||||
|
||||
void setProfilerStep(Int64 value)
|
||||
{
|
||||
profiler_step = value;
|
||||
|
@ -86,7 +86,8 @@ public:
|
||||
if (0 == needle_size)
|
||||
return;
|
||||
|
||||
UTF8SequenceBuffer l_seq, u_seq;
|
||||
UTF8SequenceBuffer l_seq;
|
||||
UTF8SequenceBuffer u_seq;
|
||||
|
||||
if (*needle < 0x80u)
|
||||
{
|
||||
@ -231,7 +232,6 @@ public:
|
||||
if (0 == needle_size)
|
||||
return haystack;
|
||||
|
||||
|
||||
while (haystack < haystack_end)
|
||||
{
|
||||
#ifdef __SSE4_1__
|
||||
|
@ -95,8 +95,11 @@ public:
|
||||
/// TODO: merge them into common entity
|
||||
ProfileEvents::Counters performance_counters{VariableContext::Thread};
|
||||
MemoryTracker memory_tracker{VariableContext::Thread};
|
||||
|
||||
/// Small amount of untracked memory (per thread atomic-less counter)
|
||||
Int64 untracked_memory = 0;
|
||||
/// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters.
|
||||
Int64 untracked_memory_limit = 4 * 1024 * 1024;
|
||||
|
||||
/// Statistics of read and write rows/bytes
|
||||
Progress progress_in;
|
||||
|
@ -56,7 +56,7 @@ TraceCollector::~TraceCollector()
|
||||
}
|
||||
|
||||
|
||||
void TraceCollector::collect(TraceType trace_type, const StackTrace & stack_trace, UInt64 size)
|
||||
void TraceCollector::collect(TraceType trace_type, const StackTrace & stack_trace, Int64 size)
|
||||
{
|
||||
constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag
|
||||
8 * sizeof(char) + // maximum VarUInt length for string size
|
||||
@ -65,7 +65,7 @@ void TraceCollector::collect(TraceType trace_type, const StackTrace & stack_trac
|
||||
sizeof(StackTrace::Frames) + // collected stack trace, maximum capacity
|
||||
sizeof(TraceType) + // trace type
|
||||
sizeof(UInt64) + // thread_id
|
||||
sizeof(UInt64); // size
|
||||
sizeof(Int64); // size
|
||||
char buffer[buf_size];
|
||||
WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer);
|
||||
|
||||
@ -139,7 +139,7 @@ void TraceCollector::run()
|
||||
UInt64 thread_id;
|
||||
readPODBinary(thread_id, in);
|
||||
|
||||
UInt64 size;
|
||||
Int64 size;
|
||||
readPODBinary(size, in);
|
||||
|
||||
if (trace_log)
|
||||
|
@ -20,6 +20,7 @@ enum class TraceType : uint8_t
|
||||
Real,
|
||||
CPU,
|
||||
Memory,
|
||||
MemorySample
|
||||
};
|
||||
|
||||
class TraceCollector
|
||||
@ -31,7 +32,7 @@ public:
|
||||
/// Collect a stack trace. This method is signal safe.
|
||||
/// Precondition: the TraceCollector object must be created.
|
||||
/// size - for memory tracing is the amount of memory allocated; for other trace types it is 0.
|
||||
static void collect(TraceType trace_type, const StackTrace & stack_trace, UInt64 size);
|
||||
static void collect(TraceType trace_type, const StackTrace & stack_trace, Int64 size);
|
||||
|
||||
private:
|
||||
std::shared_ptr<TraceLog> trace_log;
|
||||
|
@ -19,14 +19,16 @@ namespace Memory
|
||||
|
||||
inline ALWAYS_INLINE void trackMemory(std::size_t size)
|
||||
{
|
||||
std::size_t actual_size = size;
|
||||
|
||||
#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5
|
||||
/// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function
|
||||
/// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic.
|
||||
if (likely(size != 0))
|
||||
CurrentMemoryTracker::alloc(nallocx(size, 0));
|
||||
#else
|
||||
CurrentMemoryTracker::alloc(size);
|
||||
actual_size = nallocx(size, 0);
|
||||
#endif
|
||||
|
||||
CurrentMemoryTracker::alloc(actual_size);
|
||||
}
|
||||
|
||||
inline ALWAYS_INLINE bool trackMemoryNoExcept(std::size_t size) noexcept
|
||||
|
@ -342,7 +342,9 @@ struct Settings : public SettingsCollection<Settings>
|
||||
\
|
||||
M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \
|
||||
M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
|
||||
M(SettingUInt64, memory_profiler_step, 0, "Every number of bytes the memory profiler will collect the allocating stack trace. The minimal effective step is 4 MiB (less values will work as clamped to 4 MiB). Zero means disabled memory profiler.", 0) \
|
||||
M(SettingUInt64, 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(SettingUInt64, memory_profiler_step, 0, "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(SettingUInt64, 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(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \
|
||||
M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \
|
||||
|
@ -13,7 +13,6 @@
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/Native.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -151,58 +151,56 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
|
||||
}
|
||||
|
||||
auto process_it = processes.emplace(processes.end(),
|
||||
query_, client_info, settings.max_memory_usage, settings.memory_tracker_fault_probability, priorities.insert(settings.priority));
|
||||
query_, client_info, priorities.insert(settings.priority));
|
||||
|
||||
res = std::make_shared<Entry>(*this, process_it);
|
||||
|
||||
process_it->query_context = &query_context;
|
||||
|
||||
if (!client_info.current_query_id.empty())
|
||||
ProcessListForUser & user_process_list = user_to_queries[client_info.current_user];
|
||||
user_process_list.queries.emplace(client_info.current_query_id, &res->get());
|
||||
|
||||
process_it->setUserProcessList(&user_process_list);
|
||||
|
||||
/// Track memory usage for all simultaneously running queries from single user.
|
||||
user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user);
|
||||
user_process_list.user_memory_tracker.setDescription("(for user)");
|
||||
|
||||
/// Actualize thread group info
|
||||
if (auto thread_group = CurrentThread::getGroup())
|
||||
{
|
||||
ProcessListForUser & user_process_list = user_to_queries[client_info.current_user];
|
||||
user_process_list.queries.emplace(client_info.current_query_id, &res->get());
|
||||
std::lock_guard lock_thread_group(thread_group->mutex);
|
||||
thread_group->performance_counters.setParent(&user_process_list.user_performance_counters);
|
||||
thread_group->memory_tracker.setParent(&user_process_list.user_memory_tracker);
|
||||
thread_group->query = process_it->query;
|
||||
|
||||
process_it->setUserProcessList(&user_process_list);
|
||||
/// Set query-level memory trackers
|
||||
thread_group->memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage);
|
||||
|
||||
/// Track memory usage for all simultaneously running queries from single user.
|
||||
user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user);
|
||||
user_process_list.user_memory_tracker.setDescription("(for user)");
|
||||
|
||||
/// Actualize thread group info
|
||||
if (auto thread_group = CurrentThread::getGroup())
|
||||
if (query_context.hasTraceCollector())
|
||||
{
|
||||
std::lock_guard lock_thread_group(thread_group->mutex);
|
||||
thread_group->performance_counters.setParent(&user_process_list.user_performance_counters);
|
||||
thread_group->memory_tracker.setParent(&user_process_list.user_memory_tracker);
|
||||
thread_group->query = process_it->query;
|
||||
|
||||
/// Set query-level memory trackers
|
||||
thread_group->memory_tracker.setOrRaiseHardLimit(process_it->max_memory_usage);
|
||||
|
||||
if (query_context.hasTraceCollector())
|
||||
{
|
||||
/// Set up memory profiling
|
||||
thread_group->memory_tracker.setOrRaiseProfilerLimit(settings.memory_profiler_step);
|
||||
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
|
||||
}
|
||||
|
||||
thread_group->memory_tracker.setDescription("(for query)");
|
||||
if (process_it->memory_tracker_fault_probability)
|
||||
thread_group->memory_tracker.setFaultProbability(process_it->memory_tracker_fault_probability);
|
||||
|
||||
/// NOTE: Do not set the limit for thread-level memory tracker since it could show unreal values
|
||||
/// since allocation and deallocation could happen in different threads
|
||||
|
||||
process_it->thread_group = std::move(thread_group);
|
||||
/// Set up memory profiling
|
||||
thread_group->memory_tracker.setOrRaiseProfilerLimit(settings.memory_profiler_step);
|
||||
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
|
||||
thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
|
||||
}
|
||||
|
||||
if (!user_process_list.user_throttler)
|
||||
{
|
||||
if (settings.max_network_bandwidth_for_user)
|
||||
user_process_list.user_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_user, total_network_throttler);
|
||||
else if (settings.max_network_bandwidth_for_all_users)
|
||||
user_process_list.user_throttler = total_network_throttler;
|
||||
}
|
||||
thread_group->memory_tracker.setDescription("(for query)");
|
||||
if (settings.memory_tracker_fault_probability)
|
||||
thread_group->memory_tracker.setFaultProbability(settings.memory_tracker_fault_probability);
|
||||
|
||||
/// NOTE: Do not set the limit for thread-level memory tracker since it could show unreal values
|
||||
/// since allocation and deallocation could happen in different threads
|
||||
|
||||
process_it->thread_group = std::move(thread_group);
|
||||
}
|
||||
|
||||
if (!user_process_list.user_throttler)
|
||||
{
|
||||
if (settings.max_network_bandwidth_for_user)
|
||||
user_process_list.user_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_user, total_network_throttler);
|
||||
else if (settings.max_network_bandwidth_for_all_users)
|
||||
user_process_list.user_throttler = total_network_throttler;
|
||||
}
|
||||
|
||||
if (!total_network_throttler && settings.max_network_bandwidth_for_all_users)
|
||||
@ -270,16 +268,12 @@ ProcessListEntry::~ProcessListEntry()
|
||||
QueryStatus::QueryStatus(
|
||||
const String & query_,
|
||||
const ClientInfo & client_info_,
|
||||
size_t max_memory_usage_,
|
||||
double memory_tracker_fault_probability_,
|
||||
QueryPriorities::Handle && priority_handle_)
|
||||
:
|
||||
query(query_),
|
||||
client_info(client_info_),
|
||||
priority_handle(std::move(priority_handle_)),
|
||||
num_queries_increment{CurrentMetrics::Query},
|
||||
max_memory_usage(max_memory_usage_),
|
||||
memory_tracker_fault_probability(memory_tracker_fault_probability_)
|
||||
num_queries_increment{CurrentMetrics::Query}
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -99,9 +99,6 @@ protected:
|
||||
|
||||
CurrentMetrics::Increment num_queries_increment{CurrentMetrics::Query};
|
||||
|
||||
size_t max_memory_usage = 0;
|
||||
double memory_tracker_fault_probability = 0.0;
|
||||
|
||||
std::atomic<bool> is_killed { false };
|
||||
|
||||
void setUserProcessList(ProcessListForUser * user_process_list_);
|
||||
@ -132,8 +129,6 @@ public:
|
||||
QueryStatus(
|
||||
const String & query_,
|
||||
const ClientInfo & client_info_,
|
||||
size_t max_memory_usage,
|
||||
double memory_tracker_fault_probability,
|
||||
QueryPriorities::Handle && priority_handle_);
|
||||
|
||||
~QueryStatus();
|
||||
|
@ -84,9 +84,15 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_)
|
||||
query_id = query_context->getCurrentQueryId();
|
||||
initQueryProfiler();
|
||||
|
||||
const Settings & settings = query_context->getSettingsRef();
|
||||
|
||||
untracked_memory_limit = settings.max_untracked_memory;
|
||||
if (settings.memory_profiler_step && settings.memory_profiler_step < UInt64(untracked_memory_limit))
|
||||
untracked_memory_limit = settings.memory_profiler_step;
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
/// Set "nice" value if required.
|
||||
Int32 new_os_thread_priority = query_context->getSettingsRef().os_thread_priority;
|
||||
Int32 new_os_thread_priority = settings.os_thread_priority;
|
||||
if (new_os_thread_priority && hasLinuxCapability(CAP_SYS_NICE))
|
||||
{
|
||||
LOG_TRACE(log, "Setting nice to " << new_os_thread_priority);
|
||||
|
@ -16,6 +16,7 @@ const TraceDataType::Values TraceLogElement::trace_values =
|
||||
{"Real", static_cast<UInt8>(TraceType::Real)},
|
||||
{"CPU", static_cast<UInt8>(TraceType::CPU)},
|
||||
{"Memory", static_cast<UInt8>(TraceType::Memory)},
|
||||
{"MemorySample", static_cast<UInt8>(TraceType::MemorySample)},
|
||||
};
|
||||
|
||||
Block TraceLogElement::createBlock()
|
||||
@ -30,7 +31,7 @@ Block TraceLogElement::createBlock()
|
||||
{std::make_shared<DataTypeUInt64>(), "thread_id"},
|
||||
{std::make_shared<DataTypeString>(), "query_id"},
|
||||
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "trace"},
|
||||
{std::make_shared<DataTypeUInt64>(), "size"},
|
||||
{std::make_shared<DataTypeInt64>(), "size"},
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -20,7 +20,7 @@ struct TraceLogElement
|
||||
UInt64 thread_id{};
|
||||
String query_id{};
|
||||
Array trace{};
|
||||
UInt64 size{}; /// Allocation size in bytes for TraceType::Memory
|
||||
Int64 size{}; /// Allocation size in bytes for TraceType::Memory
|
||||
|
||||
static std::string name() { return "TraceLog"; }
|
||||
static Block createBlock();
|
||||
|
Loading…
Reference in New Issue
Block a user