Skip to content

Commit

Permalink
[WIP]
Browse files Browse the repository at this point in the history
  • Loading branch information
abyss7 committed Jan 21, 2020
1 parent c566f40 commit 1934706
Show file tree
Hide file tree
Showing 12 changed files with 269 additions and 148 deletions.
2 changes: 1 addition & 1 deletion .clang-format
Original file line number Diff line number Diff line change
Expand Up @@ -52,12 +52,12 @@ IncludeCategories:
ReflowComments: false
AlignEscapedNewlinesLeft: false
AlignEscapedNewlines: DontAlign
AlignTrailingComments: true

# Not changed:
AccessModifierOffset: -4
AlignConsecutiveAssignments: false
AlignOperands: false
AlignTrailingComments: false
AllowAllParametersOfDeclarationOnNextLine: true
AllowShortBlocksOnASingleLine: false
AllowShortCaseLabelsOnASingleLine: false
Expand Down
22 changes: 12 additions & 10 deletions dbms/src/Common/Allocator.h
Original file line number Diff line number Diff line change
Expand Up @@ -93,8 +93,9 @@ class Allocator
/// Allocate memory range.
void * alloc(size_t size, size_t alignment = 0)
{
auto * ptr = allocNoTrack(size, alignment);
CurrentMemoryTracker::alloc(size);
return allocNoTrack(size, alignment);
return ptr;
}

/// Free memory range.
Expand All @@ -118,39 +119,40 @@ class Allocator
else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD
&& alignment <= MALLOC_MIN_ALIGNMENT)
{
/// Resize malloc'd memory region with no special alignment requirement.
CurrentMemoryTracker::realloc(old_size, new_size);

void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);

/// Resize malloc'd memory region with no special alignment requirement.
CurrentMemoryTracker::realloc(old_size, new_size);

buf = new_buf;
if constexpr (clear_memory)
if (new_size > old_size)
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
}
else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
{
/// Resize mmap'd memory region.
CurrentMemoryTracker::realloc(old_size, new_size);

// On apple and freebsd self-implemented mremap used (common/mremap.h)
buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE,
PROT_READ | PROT_WRITE, mmap_flags, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP);

/// Resize mmap'd memory region.
CurrentMemoryTracker::realloc(old_size, new_size);

/// No need for zero-fill, because mmap guarantees it.
}
else if (new_size < MMAP_THRESHOLD)
{
/// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once.
CurrentMemoryTracker::realloc(old_size, new_size);

void * new_buf = allocNoTrack(new_size, alignment);
memcpy(new_buf, buf, std::min(old_size, new_size));
freeNoTrack(buf, old_size);

/// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once.
CurrentMemoryTracker::realloc(old_size, new_size);

buf = new_buf;
}
else
Expand Down
27 changes: 15 additions & 12 deletions dbms/src/Common/MemoryTracker.cpp
Original file line number Diff line number Diff line change
@@ -1,12 +1,15 @@
#include <cstdlib>

#include "MemoryTracker.h"
#include <common/likely.h>
#include <common/logger_useful.h>

#include <IO/WriteHelpers.h>
#include "Common/TraceCollector.h"
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
#include <Common/CurrentThread.h>
#include <IO/WriteHelpers.h>
#include <common/likely.h>
#include <common/logger_useful.h>
#include <common/singleton.h>

#include <cstdlib>


namespace DB
Expand Down Expand Up @@ -73,7 +76,7 @@ void MemoryTracker::alloc(Int64 size)
return;

/** Using memory_order_relaxed means that if allocations are done simultaneously,
* we allow exception about memory limit exceeded to be thrown only on next allocation.
* we allow exception about memory limit exceeded to be thrown only on next allocation.
* So, we allow over-allocations.
*/
Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed);
Expand Down Expand Up @@ -207,21 +210,21 @@ namespace CurrentMemoryTracker
if (untracked > 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 usefull for enlarge Exception message in rethrow logic.
/// more. It could be useful to enlarge Exception message in rethrow logic.
Int64 tmp = untracked;
untracked = 0;
memory_tracker->alloc(tmp);

auto no_track = memory_tracker->blocker.cancel();
Singleton<DB::TraceCollector>()->collect(tmp);
}
}
}

void realloc(Int64 old_size, Int64 new_size)
{
Int64 addition = new_size - old_size;
if (addition > 0)
alloc(addition);
else
free(-addition);
addition > 0 ? alloc(addition) : free(-addition);
}

void free(Int64 size)
Expand Down
80 changes: 13 additions & 67 deletions dbms/src/Common/QueryProfiler.cpp
Original file line number Diff line number Diff line change
@@ -1,92 +1,38 @@
#include "QueryProfiler.h"

#include <random>
#include <common/phdr_cache.h>
#include <common/config_common.h>
#include <common/StringRef.h>
#include <common/logger_useful.h>
#include <Common/PipeFDs.h>
#include <Common/StackTrace.h>
#include <Common/CurrentThread.h>
#include <IO/WriteHelpers.h>
#include <Common/Exception.h>
#include <Common/StackTrace.h>
#include <Common/TraceCollector.h>
#include <Common/thread_local_rng.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
#include <common/StringRef.h>
#include <common/config_common.h>
#include <common/logger_useful.h>
#include <common/phdr_cache.h>
#include <common/singleton.h>

#include <random>

namespace ProfileEvents
{
extern const Event QueryProfilerSignalOverruns;
}

namespace DB
{

extern LazyPipeFDs trace_pipe;

namespace
{
/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id.
/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler.
constexpr size_t QUERY_ID_MAX_LEN = 1024;

#if defined(OS_LINUX)
thread_local size_t write_trace_iteration = 0;
#endif

void writeTraceInfo(TraceType trace_type, int /* sig */, siginfo_t * info, void * context)
{
int overrun_count = 0;
#if defined(OS_LINUX)
/// Quickly drop if signal handler is called too frequently.
/// Otherwise we may end up infinitelly processing signals instead of doing any useful work.
++write_trace_iteration;
if (info && info->si_overrun > 0)
{
/// But pass with some frequency to avoid drop of all traces.
if (write_trace_iteration % info->si_overrun == 0)
{
ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, info->si_overrun);
}
else
{
ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, info->si_overrun + 1);
return;
}
}
if (info)
overrun_count = info->si_overrun;
#else
UNUSED(info);
#endif

constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag
8 * sizeof(char) + // maximum VarUInt length for string size
QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length
sizeof(UInt8) + // number of stack frames
sizeof(StackTrace::Frames) + // collected stack trace, maximum capacity
sizeof(TraceType) + // timer type
sizeof(UInt32); // thread_number
char buffer[buf_size];
WriteBufferFromFileDescriptorDiscardOnFailure out(trace_pipe.fds_rw[1], buf_size, buffer);

StringRef query_id = CurrentThread::getQueryId();
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);

UInt32 thread_number = CurrentThread::get().thread_number;

const auto signal_context = *reinterpret_cast<ucontext_t *>(context);
const StackTrace stack_trace(signal_context);

writeChar(false, out);
writeStringBinary(query_id, out);

size_t stack_trace_size = stack_trace.getSize();
size_t stack_trace_offset = stack_trace.getOffset();
writeIntBinary(UInt8(stack_trace_size - stack_trace_offset), out);
for (size_t i = stack_trace_offset; i < stack_trace_size; ++i)
writePODBinary(stack_trace.getFrames()[i], out);

writePODBinary(trace_type, out);
writePODBinary(thread_number, out);
out.next();
Singleton<TraceCollector>()->collect(trace_type, stack_trace, overrun_count);
}

[[maybe_unused]] const UInt32 TIMER_PRECISION = 1e9;
Expand Down
7 changes: 0 additions & 7 deletions dbms/src/Common/QueryProfiler.h
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,6 @@ namespace Poco
namespace DB
{

enum class TraceType : UInt8
{
REAL_TIME,
CPU_TIME,
MEMORY,
};

/**
* Query profiler implementation for selected thread.
*
Expand Down
Loading

0 comments on commit 1934706

Please sign in to comment.