2018-06-20 17:49:52 +00:00
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <Common/ThreadProfileEvents.h>
|
2019-03-04 13:03:32 +00:00
|
|
|
#include <Common/QueryProfiler.h>
|
2018-08-22 00:24:55 +00:00
|
|
|
#include <Common/ThreadStatus.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/errnoToString.h>
|
2020-11-09 15:07:38 +00:00
|
|
|
#include <Interpreters/OpenTelemetrySpanLog.h>
|
2018-05-17 16:01:41 +00:00
|
|
|
|
2018-08-22 00:24:55 +00:00
|
|
|
#include <Poco/Logger.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/getThreadId.h>
|
|
|
|
#include <base/getPageSize.h>
|
2018-02-01 17:55:08 +00:00
|
|
|
|
2021-06-20 07:00:15 +00:00
|
|
|
#include <csignal>
|
2021-09-02 14:27:19 +00:00
|
|
|
#include <mutex>
|
2021-06-19 14:36:25 +00:00
|
|
|
|
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-05-29 18:14:31 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
2018-09-16 00:14:55 +00:00
|
|
|
|
2019-02-08 13:23:10 +00:00
|
|
|
thread_local ThreadStatus * current_thread = nullptr;
|
Use total_memory_tracker when there is no other MemoryTracker object.
This should significantly reduce the MemoryTracking drift, test shows
that there is 0 drift after query storm (100 queries, via http/tcp/tcp
in one session).
TL;DR;
To track memory, clickhouse creates memory tracker object for each
thread **explicitly**, but until it is not created the memory
allocations are not under account.
There should not be lot of allocations w/o memory tracker, since most of
the time it is created early enough, but even this maybe enough to
trigger some problems.
Plus sometimes it is not possible to create it, for example some 3d
party library does not allow to do this explicitly:
- for example before #15740 allocations from librdkafka threads,
- or even worse, poco threads, they don't have any routines to do this.
This won't be a problem for `MemoryTracking` metric if the deallocation
will be done from the same thread w/o memory tracker (or vise versa),
but this is not always true.
NOTE, that this will slow down per-thread allocations w/o memory
tracker, since before this patch there were no memory tracking for them
while now they will be accounted in total_memory_tracker, and for
total_memory_tracker max_untracked_memory is always reached.
But this should not be significant.
2020-10-18 07:32:49 +00:00
|
|
|
thread_local ThreadStatus * main_thread = nullptr;
|
2019-01-13 18:51:57 +00:00
|
|
|
|
2021-10-27 23:10:39 +00:00
|
|
|
#if !defined(SANITIZER)
|
2021-06-19 14:36:25 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2021-06-20 07:00:15 +00:00
|
|
|
/// Alternative stack for signal handling.
|
|
|
|
///
|
|
|
|
/// This stack should not be located in the TLS (thread local storage), since:
|
|
|
|
/// - TLS locates data on the per-thread stack
|
|
|
|
/// - And in case of stack in the signal handler will grow too much,
|
|
|
|
/// it will start overwriting TLS storage
|
|
|
|
/// (and note, that it is not too small, due to StackTrace obtaining)
|
|
|
|
/// - Plus there is no way to determine TLS block size, yes there is
|
|
|
|
/// __pthread_get_minstack() in glibc, but it is private and hence not portable.
|
|
|
|
///
|
|
|
|
/// Also we should not use getStackSize() (pthread_attr_getstack()) since it
|
|
|
|
/// will return 8MB, and this is too huge for signal stack.
|
2021-06-19 14:36:25 +00:00
|
|
|
struct ThreadStack
|
|
|
|
{
|
|
|
|
ThreadStack()
|
2021-10-06 21:17:14 +00:00
|
|
|
: data(aligned_alloc(getPageSize(), getSize()))
|
2021-06-20 16:54:24 +00:00
|
|
|
{
|
|
|
|
/// Add a guard page
|
|
|
|
/// (and since the stack grows downward, we need to protect the first page).
|
|
|
|
mprotect(data, getPageSize(), PROT_NONE);
|
|
|
|
}
|
2021-06-19 14:36:25 +00:00
|
|
|
~ThreadStack()
|
|
|
|
{
|
2021-06-20 16:54:24 +00:00
|
|
|
mprotect(data, getPageSize(), PROT_WRITE|PROT_READ);
|
2021-06-19 14:36:25 +00:00
|
|
|
free(data);
|
|
|
|
}
|
|
|
|
|
2021-10-06 21:17:14 +00:00
|
|
|
static size_t getSize() { return std::max<size_t>(16 << 10, MINSIGSTKSZ); }
|
2021-06-21 09:06:41 +00:00
|
|
|
void * getData() const { return data; }
|
2021-06-19 14:36:25 +00:00
|
|
|
|
|
|
|
private:
|
2021-06-23 21:32:34 +00:00
|
|
|
/// 16 KiB - not too big but enough to handle error.
|
2021-06-21 09:06:41 +00:00
|
|
|
void * data;
|
2021-06-19 14:36:25 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
static thread_local ThreadStack alt_stack;
|
|
|
|
static thread_local bool has_alt_stack = false;
|
2020-10-26 02:18:36 +00:00
|
|
|
#endif
|
2020-10-25 00:10:05 +00:00
|
|
|
|
2019-01-13 18:51:57 +00:00
|
|
|
|
2021-12-14 07:25:30 +00:00
|
|
|
std::vector<ThreadGroupStatus::ProfileEventsCountersAndMemory> ThreadGroupStatus::getProfileEventsCountersAndMemoryForThreads()
|
|
|
|
{
|
|
|
|
std::lock_guard guard(mutex);
|
|
|
|
|
|
|
|
/// It is OK to move it, since it is enough to report statistics for the thread at least once.
|
|
|
|
auto stats = std::move(finished_threads_counters_memory);
|
|
|
|
for (auto * thread : threads)
|
|
|
|
{
|
|
|
|
stats.emplace_back(ProfileEventsCountersAndMemory{
|
|
|
|
thread->performance_counters.getPartiallyAtomicSnapshot(),
|
|
|
|
thread->memory_tracker.get(),
|
|
|
|
thread->thread_id,
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
return stats;
|
|
|
|
}
|
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
ThreadStatus::ThreadStatus()
|
2020-05-01 18:47:41 +00:00
|
|
|
: thread_id{getThreadId()}
|
2018-02-01 17:55:08 +00:00
|
|
|
{
|
2018-08-14 20:29:42 +00:00
|
|
|
last_rusage = std::make_unique<RUsageCounters>();
|
2018-03-01 16:52:24 +00:00
|
|
|
|
2018-06-09 15:29:08 +00:00
|
|
|
memory_tracker.setDescription("(for thread)");
|
|
|
|
log = &Poco::Logger::get("ThreadStatus");
|
|
|
|
|
2019-01-13 18:51:57 +00:00
|
|
|
current_thread = this;
|
|
|
|
|
2018-06-06 20:57:07 +00:00
|
|
|
/// NOTE: It is important not to do any non-trivial actions (like updating ProfileEvents or logging) before ThreadStatus is created
|
|
|
|
/// Otherwise it could lead to SIGSEGV due to current_thread dereferencing
|
2020-10-25 00:10:05 +00:00
|
|
|
|
|
|
|
/// Will set alternative signal stack to provide diagnostics for stack overflow errors.
|
|
|
|
/// If not already installed for current thread.
|
2020-10-26 19:46:25 +00:00
|
|
|
/// Sanitizer makes larger stack usage and also it's incompatible with alternative stack by default (it sets up and relies on its own).
|
2021-10-27 23:10:39 +00:00
|
|
|
#if !defined(SANITIZER)
|
2020-10-25 00:10:05 +00:00
|
|
|
if (!has_alt_stack)
|
|
|
|
{
|
|
|
|
/// Don't repeat tries even if not installed successfully.
|
|
|
|
has_alt_stack = true;
|
|
|
|
|
|
|
|
/// We have to call 'sigaltstack' before first 'sigaction'. (It does not work other way, for unknown reason).
|
2020-10-26 04:42:22 +00:00
|
|
|
stack_t altstack_description{};
|
2021-06-19 14:36:25 +00:00
|
|
|
altstack_description.ss_sp = alt_stack.getData();
|
2020-10-26 04:42:22 +00:00
|
|
|
altstack_description.ss_flags = 0;
|
2021-06-19 14:36:25 +00:00
|
|
|
altstack_description.ss_size = alt_stack.getSize();
|
2020-10-26 04:42:22 +00:00
|
|
|
|
2020-10-25 00:10:05 +00:00
|
|
|
if (0 != sigaltstack(&altstack_description, nullptr))
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Cannot set alternative signal stack for thread, {}", errnoToString(errno));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Obtain existing sigaction and modify it by adding a flag.
|
|
|
|
struct sigaction action{};
|
|
|
|
if (0 != sigaction(SIGSEGV, nullptr, &action))
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Cannot obtain previous signal action to set alternative signal stack for thread, {}", errnoToString(errno));
|
|
|
|
}
|
|
|
|
else if (!(action.sa_flags & SA_ONSTACK))
|
|
|
|
{
|
|
|
|
action.sa_flags |= SA_ONSTACK;
|
|
|
|
|
|
|
|
if (0 != sigaction(SIGSEGV, &action, nullptr))
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Cannot set action with alternative signal stack for thread, {}", errnoToString(errno));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-10-26 02:18:36 +00:00
|
|
|
#endif
|
2018-05-29 18:14:31 +00:00
|
|
|
}
|
2018-03-01 16:52:24 +00:00
|
|
|
|
2019-01-13 18:51:57 +00:00
|
|
|
ThreadStatus::~ThreadStatus()
|
2018-05-29 18:14:31 +00:00
|
|
|
{
|
2019-07-16 11:56:46 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
if (untracked_memory > 0)
|
|
|
|
memory_tracker.alloc(untracked_memory);
|
|
|
|
else
|
|
|
|
memory_tracker.free(-untracked_memory);
|
|
|
|
}
|
|
|
|
catch (const DB::Exception &)
|
|
|
|
{
|
|
|
|
/// It's a minor tracked memory leak here (not the memory itself but it's counter).
|
2020-12-29 10:16:22 +00:00
|
|
|
/// We've already allocated a little bit more than the limit and cannot track it in the thread memory tracker or its parent.
|
2021-12-14 07:25:30 +00:00
|
|
|
tryLogCurrentException(log);
|
2019-07-16 11:56:46 +00:00
|
|
|
}
|
2019-07-10 18:12:50 +00:00
|
|
|
|
2021-09-07 12:07:24 +00:00
|
|
|
if (thread_group)
|
|
|
|
{
|
|
|
|
std::lock_guard guard(thread_group->mutex);
|
2021-12-14 07:25:30 +00:00
|
|
|
thread_group->finished_threads_counters_memory.emplace_back(ThreadGroupStatus::ProfileEventsCountersAndMemory{
|
|
|
|
performance_counters.getPartiallyAtomicSnapshot(),
|
|
|
|
memory_tracker.get(),
|
|
|
|
thread_id,
|
|
|
|
});
|
2021-09-07 12:07:24 +00:00
|
|
|
thread_group->threads.erase(this);
|
|
|
|
}
|
|
|
|
|
2021-01-28 13:57:36 +00:00
|
|
|
/// It may cause segfault if query_context was destroyed, but was not detached
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context_ptr = query_context.lock();
|
|
|
|
assert((!query_context_ptr && query_id.empty()) || (query_context_ptr && query_id == query_context_ptr->getCurrentQueryId()));
|
2021-01-28 13:57:36 +00:00
|
|
|
|
2019-01-13 18:51:57 +00:00
|
|
|
if (deleter)
|
|
|
|
deleter();
|
2021-08-09 17:38:29 +00:00
|
|
|
|
|
|
|
/// Only change current_thread if it's currently being used by this ThreadStatus
|
|
|
|
/// For example, PushingToViewsBlockOutputStream creates and deletes ThreadStatus instances while running in the main query thread
|
2021-06-23 14:33:23 +00:00
|
|
|
if (current_thread == this)
|
|
|
|
current_thread = nullptr;
|
2018-02-01 17:55:08 +00:00
|
|
|
}
|
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
void ThreadStatus::updatePerformanceCounters()
|
2018-02-01 17:55:08 +00:00
|
|
|
{
|
2018-05-31 15:54:08 +00:00
|
|
|
try
|
|
|
|
{
|
2018-08-14 20:29:42 +00:00
|
|
|
RUsageCounters::updateProfileEvents(*last_rusage, performance_counters);
|
2020-05-01 18:47:41 +00:00
|
|
|
if (taskstats)
|
|
|
|
taskstats->updateCounters(performance_counters);
|
2018-05-31 15:54:08 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log);
|
|
|
|
}
|
2018-06-01 19:39:32 +00:00
|
|
|
}
|
|
|
|
|
2020-04-22 05:39:31 +00:00
|
|
|
void ThreadStatus::assertState(const std::initializer_list<int> & permitted_states, const char * description) const
|
2018-03-01 16:52:24 +00:00
|
|
|
{
|
2018-06-19 20:30:35 +00:00
|
|
|
for (auto permitted_state : permitted_states)
|
2018-06-01 19:39:32 +00:00
|
|
|
{
|
2018-06-19 20:30:35 +00:00
|
|
|
if (getCurrentState() == permitted_state)
|
|
|
|
return;
|
2018-06-01 19:39:32 +00:00
|
|
|
}
|
2018-05-31 15:54:08 +00:00
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
if (description)
|
2020-11-10 18:22:26 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", getCurrentState(), description);
|
|
|
|
else
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}", getCurrentState());
|
2018-06-19 20:30:35 +00:00
|
|
|
}
|
|
|
|
|
2019-07-09 10:39:05 +00:00
|
|
|
void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue,
|
|
|
|
LogsLevel client_logs_level)
|
2018-06-19 20:30:35 +00:00
|
|
|
{
|
|
|
|
logs_queue_ptr = logs_queue;
|
|
|
|
|
|
|
|
if (!thread_group)
|
|
|
|
return;
|
|
|
|
|
2019-02-08 13:23:10 +00:00
|
|
|
std::lock_guard lock(thread_group->mutex);
|
2018-06-19 20:30:35 +00:00
|
|
|
thread_group->logs_queue_ptr = logs_queue;
|
2019-07-09 10:39:05 +00:00
|
|
|
thread_group->client_logs_level = client_logs_level;
|
2018-06-19 20:30:35 +00:00
|
|
|
}
|
|
|
|
|
2021-09-02 14:27:19 +00:00
|
|
|
void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue)
|
|
|
|
{
|
|
|
|
profile_queue_ptr = profile_queue;
|
|
|
|
|
|
|
|
if (!thread_group)
|
|
|
|
return;
|
|
|
|
|
|
|
|
std::lock_guard lock(thread_group->mutex);
|
|
|
|
thread_group->profile_queue_ptr = profile_queue;
|
|
|
|
}
|
|
|
|
|
2020-06-20 11:17:15 +00:00
|
|
|
void ThreadStatus::setFatalErrorCallback(std::function<void()> callback)
|
|
|
|
{
|
|
|
|
fatal_error_callback = std::move(callback);
|
2020-07-09 04:15:45 +00:00
|
|
|
|
|
|
|
if (!thread_group)
|
|
|
|
return;
|
|
|
|
|
|
|
|
std::lock_guard lock(thread_group->mutex);
|
|
|
|
thread_group->fatal_error_callback = fatal_error_callback;
|
2020-06-20 11:17:15 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadStatus::onFatalError()
|
|
|
|
{
|
|
|
|
if (fatal_error_callback)
|
|
|
|
fatal_error_callback();
|
|
|
|
}
|
|
|
|
|
Use total_memory_tracker when there is no other MemoryTracker object.
This should significantly reduce the MemoryTracking drift, test shows
that there is 0 drift after query storm (100 queries, via http/tcp/tcp
in one session).
TL;DR;
To track memory, clickhouse creates memory tracker object for each
thread **explicitly**, but until it is not created the memory
allocations are not under account.
There should not be lot of allocations w/o memory tracker, since most of
the time it is created early enough, but even this maybe enough to
trigger some problems.
Plus sometimes it is not possible to create it, for example some 3d
party library does not allow to do this explicitly:
- for example before #15740 allocations from librdkafka threads,
- or even worse, poco threads, they don't have any routines to do this.
This won't be a problem for `MemoryTracking` metric if the deallocation
will be done from the same thread w/o memory tracker (or vise versa),
but this is not always true.
NOTE, that this will slow down per-thread allocations w/o memory
tracker, since before this patch there were no memory tracking for them
while now they will be accounted in total_memory_tracker, and for
total_memory_tracker max_untracked_memory is always reached.
But this should not be significant.
2020-10-18 07:32:49 +00:00
|
|
|
ThreadStatus * MainThreadStatus::main_thread = nullptr;
|
|
|
|
MainThreadStatus & MainThreadStatus::getInstance()
|
|
|
|
{
|
|
|
|
static MainThreadStatus thread_status;
|
|
|
|
return thread_status;
|
|
|
|
}
|
|
|
|
MainThreadStatus::MainThreadStatus()
|
|
|
|
{
|
|
|
|
main_thread = current_thread;
|
|
|
|
}
|
|
|
|
MainThreadStatus::~MainThreadStatus()
|
|
|
|
{
|
|
|
|
main_thread = nullptr;
|
|
|
|
}
|
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
}
|