2018-06-20 17:49:52 +00:00
|
|
|
#include <Common/ThreadStatus.h>
|
|
|
|
#include <Common/CurrentThread.h>
|
|
|
|
#include <Common/ThreadProfileEvents.h>
|
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/QueryThreadLog.h>
|
|
|
|
#include <Interpreters/ProcessList.h>
|
2019-02-03 09:57:12 +00:00
|
|
|
#include <Interpreters/TraceCollector.h>
|
|
|
|
#include <IO/WriteBufferFromFileDescriptor.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2018-12-25 00:19:17 +00:00
|
|
|
#include <common/logger_useful.h>
|
2018-06-20 17:49:52 +00:00
|
|
|
|
2018-12-25 00:19:17 +00:00
|
|
|
#include <csignal>
|
|
|
|
#include <time.h>
|
|
|
|
#include <signal.h>
|
|
|
|
#include <sys/syscall.h>
|
2019-02-03 09:57:12 +00:00
|
|
|
#include <unistd.h>
|
2018-06-20 17:49:52 +00:00
|
|
|
|
|
|
|
/// Implement some methods of ThreadStatus and CurrentThread here to avoid extra linking dependencies in clickhouse_common_io
|
2019-02-04 15:39:08 +00:00
|
|
|
/// TODO It doesn't make sense.
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
void ThreadStatus::attachQueryContext(Context & query_context_)
|
|
|
|
{
|
|
|
|
query_context = &query_context_;
|
|
|
|
if (!global_context)
|
|
|
|
global_context = &query_context->getGlobalContext();
|
|
|
|
|
2019-02-04 15:39:08 +00:00
|
|
|
query_id = query_context->getCurrentQueryId();
|
2018-06-20 17:49:52 +00:00
|
|
|
|
2019-02-04 15:39:08 +00:00
|
|
|
if (thread_group)
|
|
|
|
{
|
2019-02-08 13:23:10 +00:00
|
|
|
std::lock_guard lock(thread_group->mutex);
|
2019-02-04 15:39:08 +00:00
|
|
|
thread_group->query_context = query_context;
|
|
|
|
if (!thread_group->global_context)
|
|
|
|
thread_group->global_context = global_context;
|
|
|
|
}
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2019-02-04 15:39:08 +00:00
|
|
|
const std::string & ThreadStatus::getQueryId() const
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2019-02-04 15:39:08 +00:00
|
|
|
return query_id;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2018-09-05 21:01:43 +00:00
|
|
|
void CurrentThread::defaultThreadDeleter()
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
ThreadStatus & thread = CurrentThread::get();
|
2018-06-20 17:49:52 +00:00
|
|
|
thread.detachQuery(true, true);
|
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadStatus::initializeQuery()
|
|
|
|
{
|
|
|
|
assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__);
|
|
|
|
|
|
|
|
thread_group = std::make_shared<ThreadGroupStatus>();
|
|
|
|
|
|
|
|
performance_counters.setParent(&thread_group->performance_counters);
|
|
|
|
memory_tracker.setParent(&thread_group->memory_tracker);
|
|
|
|
thread_group->memory_tracker.setDescription("(for query)");
|
|
|
|
|
2019-02-08 13:23:10 +00:00
|
|
|
thread_group->thread_numbers.emplace_back(thread_number);
|
|
|
|
thread_group->master_thread_number = thread_number;
|
|
|
|
thread_group->master_thread_os_id = os_thread_id;
|
2018-06-20 17:49:52 +00:00
|
|
|
|
|
|
|
initPerformanceCounters();
|
|
|
|
thread_state = ThreadState::AttachedToQuery;
|
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached)
|
|
|
|
{
|
|
|
|
if (thread_state == ThreadState::AttachedToQuery)
|
|
|
|
{
|
|
|
|
if (check_detached)
|
|
|
|
throw Exception("Can't attach query to the thread, it is already attached", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__);
|
|
|
|
|
|
|
|
if (!thread_group_)
|
|
|
|
throw Exception("Attempt to attach to nullptr thread group", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
/// Attach current thread to thread group and copy useful information from it
|
|
|
|
thread_group = thread_group_;
|
|
|
|
|
|
|
|
performance_counters.setParent(&thread_group->performance_counters);
|
|
|
|
memory_tracker.setParent(&thread_group->memory_tracker);
|
|
|
|
|
|
|
|
{
|
2019-02-08 13:23:10 +00:00
|
|
|
std::lock_guard lock(thread_group->mutex);
|
2018-06-20 17:49:52 +00:00
|
|
|
|
|
|
|
logs_queue_ptr = thread_group->logs_queue_ptr;
|
|
|
|
query_context = thread_group->query_context;
|
|
|
|
|
|
|
|
if (!global_context)
|
|
|
|
global_context = thread_group->global_context;
|
|
|
|
|
2019-01-25 14:08:01 +00:00
|
|
|
/// NOTE: A thread may be attached multiple times if it is reused from a thread pool.
|
2019-02-08 13:23:10 +00:00
|
|
|
thread_group->thread_numbers.emplace_back(thread_number);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
initPerformanceCounters();
|
2018-12-25 00:19:17 +00:00
|
|
|
initQueryProfiler();
|
2019-01-18 13:10:43 +00:00
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
thread_state = ThreadState::AttachedToQuery;
|
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadStatus::finalizePerformanceCounters()
|
|
|
|
{
|
|
|
|
if (performance_counters_finalized)
|
|
|
|
return;
|
|
|
|
|
|
|
|
performance_counters_finalized = true;
|
|
|
|
updatePerformanceCounters();
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
2018-09-28 03:40:09 +00:00
|
|
|
if (global_context && query_context)
|
|
|
|
{
|
|
|
|
auto & settings = query_context->getSettingsRef();
|
|
|
|
if (settings.log_queries && settings.log_query_threads)
|
|
|
|
if (auto thread_log = global_context->getQueryThreadLog())
|
|
|
|
logToQueryThreadLog(*thread_log);
|
|
|
|
}
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
void queryProfilerTimerHandler(int /* sig */, siginfo_t * /* info */, void * context)
|
|
|
|
{
|
2019-02-09 21:40:10 +00:00
|
|
|
DB::WriteBufferFromFileDescriptor out(trace_pipe.fds_rw[1]);
|
2019-02-03 09:57:12 +00:00
|
|
|
|
2019-02-09 22:26:51 +00:00
|
|
|
const std::string & query_id = CurrentThread::getQueryId();
|
2019-02-03 09:57:12 +00:00
|
|
|
|
2019-02-03 21:30:45 +00:00
|
|
|
DB::writePODBinary(*reinterpret_cast<const ucontext_t *>(context), out);
|
2019-02-09 21:40:10 +00:00
|
|
|
DB::writeStringBinary(query_id, out);
|
2019-02-03 09:57:12 +00:00
|
|
|
out.next();
|
2018-12-25 00:19:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
void ThreadStatus::initQueryProfiler()
|
|
|
|
{
|
|
|
|
if (!query_context)
|
2019-01-18 13:10:43 +00:00
|
|
|
return;
|
|
|
|
|
|
|
|
struct sigevent sev;
|
2018-12-25 00:19:17 +00:00
|
|
|
sev.sigev_notify = SIGEV_THREAD_ID;
|
2019-02-03 21:30:45 +00:00
|
|
|
sev.sigev_signo = pause_signal;
|
2018-12-25 00:19:17 +00:00
|
|
|
sev._sigev_un._tid = os_thread_id;
|
|
|
|
// TODO(laplab): get clock type from settings
|
2019-02-09 22:40:47 +00:00
|
|
|
if (timer_create(CLOCK_REALTIME, &sev, &query_profiler_timer_id))
|
2019-01-18 13:10:43 +00:00
|
|
|
throw Poco::Exception("Failed to create query profiler timer");
|
|
|
|
|
2018-12-25 00:19:17 +00:00
|
|
|
// TODO(laplab): get period from settings
|
2019-01-18 13:10:43 +00:00
|
|
|
struct timespec period{.tv_sec = 0, .tv_nsec = 200000000};
|
|
|
|
struct itimerspec timer_spec = {.it_interval = period, .it_value = period};
|
2019-02-09 22:40:47 +00:00
|
|
|
if (timer_settime(query_profiler_timer_id, 0, &timer_spec, nullptr))
|
2019-01-18 13:10:43 +00:00
|
|
|
throw Poco::Exception("Failed to set query profiler timer");
|
2018-12-25 00:19:17 +00:00
|
|
|
|
2019-02-09 21:40:10 +00:00
|
|
|
struct sigaction sa{};
|
2019-01-18 13:10:43 +00:00
|
|
|
sa.sa_sigaction = queryProfilerTimerHandler;
|
|
|
|
sa.sa_flags = SA_SIGINFO;
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
if (sigemptyset(&sa.sa_mask))
|
2019-01-18 13:10:43 +00:00
|
|
|
throw Poco::Exception("Failed to clean signal mask for query profiler");
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
if (sigaddset(&sa.sa_mask, pause_signal))
|
2019-01-18 13:10:43 +00:00
|
|
|
throw Poco::Exception("Failed to add signal to mask for query profiler");
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
if (sigaction(pause_signal, &sa, previous_handler))
|
2019-01-18 13:10:43 +00:00
|
|
|
throw Poco::Exception("Failed to setup signal handler for query profiler");
|
2018-12-25 00:19:17 +00:00
|
|
|
|
|
|
|
has_query_profiler = true;
|
|
|
|
}
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
void ThreadStatus::finalizeQueryProfiler()
|
|
|
|
{
|
|
|
|
if (!has_query_profiler)
|
2018-12-25 00:19:17 +00:00
|
|
|
return;
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
if (timer_delete(query_profiler_timer_id))
|
2019-01-18 13:10:43 +00:00
|
|
|
throw Poco::Exception("Failed to delete query profiler timer");
|
2018-12-25 00:19:17 +00:00
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
if (sigaction(pause_signal, previous_handler, nullptr))
|
2019-02-09 21:40:10 +00:00
|
|
|
throw Poco::Exception("Failed to restore signal handler after query profiler");
|
|
|
|
|
2018-12-25 00:19:17 +00:00
|
|
|
has_query_profiler = false;
|
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
|
|
|
|
{
|
|
|
|
if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery)
|
|
|
|
{
|
|
|
|
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__);
|
2018-12-25 00:19:17 +00:00
|
|
|
|
|
|
|
finalizeQueryProfiler();
|
2018-06-20 17:49:52 +00:00
|
|
|
finalizePerformanceCounters();
|
|
|
|
|
|
|
|
/// Detach from thread group
|
|
|
|
performance_counters.setParent(&ProfileEvents::global_counters);
|
2018-09-26 20:45:40 +00:00
|
|
|
memory_tracker.reset();
|
2018-10-05 19:29:49 +00:00
|
|
|
|
|
|
|
/// Must reset pointer to thread_group's memory_tracker, because it will be destroyed two lines below.
|
|
|
|
memory_tracker.setParent(nullptr);
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
query_context = nullptr;
|
|
|
|
thread_group.reset();
|
|
|
|
|
|
|
|
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;
|
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log)
|
|
|
|
{
|
|
|
|
QueryThreadLogElement elem;
|
|
|
|
|
|
|
|
elem.event_time = time(nullptr);
|
|
|
|
elem.query_start_time = query_start_time;
|
|
|
|
elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U;
|
|
|
|
|
|
|
|
elem.read_rows = progress_in.rows.load(std::memory_order_relaxed);
|
|
|
|
elem.read_bytes = progress_in.bytes.load(std::memory_order_relaxed);
|
|
|
|
elem.written_rows = progress_out.rows.load(std::memory_order_relaxed);
|
|
|
|
elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed);
|
|
|
|
elem.memory_usage = memory_tracker.get();
|
|
|
|
elem.peak_memory_usage = memory_tracker.getPeak();
|
|
|
|
|
|
|
|
elem.thread_name = getThreadName();
|
|
|
|
elem.thread_number = thread_number;
|
|
|
|
elem.os_thread_id = os_thread_id;
|
|
|
|
|
|
|
|
if (thread_group)
|
|
|
|
{
|
|
|
|
{
|
2019-02-08 13:23:10 +00:00
|
|
|
std::lock_guard lock(thread_group->mutex);
|
2018-06-20 17:49:52 +00:00
|
|
|
|
2019-02-08 13:23:10 +00:00
|
|
|
elem.master_thread_number = thread_group->master_thread_number;
|
|
|
|
elem.master_os_thread_id = thread_group->master_thread_os_id;
|
2018-06-20 17:49:52 +00:00
|
|
|
|
|
|
|
elem.query = thread_group->query;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (query_context)
|
|
|
|
{
|
|
|
|
elem.client_info = query_context->getClientInfo();
|
|
|
|
|
|
|
|
if (query_context->getSettingsRef().log_profile_events.value != 0)
|
|
|
|
{
|
|
|
|
/// NOTE: Here we are in the same thread, so we can make memcpy()
|
|
|
|
elem.profile_counters = std::make_shared<ProfileEvents::Counters>(performance_counters.getPartiallyAtomicSnapshot());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
thread_log.add(elem);
|
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::initializeQuery()
|
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
get().initializeQuery();
|
|
|
|
get().deleter = CurrentThread::defaultThreadDeleter;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group)
|
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
get().attachQuery(thread_group, true);
|
|
|
|
get().deleter = CurrentThread::defaultThreadDeleter;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group)
|
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
get().attachQuery(thread_group, false);
|
|
|
|
get().deleter = CurrentThread::defaultThreadDeleter;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2019-02-04 15:39:08 +00:00
|
|
|
const std::string & CurrentThread::getQueryId()
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2019-02-04 15:39:08 +00:00
|
|
|
return get().getQueryId();
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::attachQueryContext(Context & query_context)
|
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
return get().attachQueryContext(query_context);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::finalizePerformanceCounters()
|
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
get().finalizePerformanceCounters();
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::detachQuery()
|
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
get().detachQuery(false);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::detachQueryIfNotDetached()
|
|
|
|
{
|
2019-01-13 18:51:57 +00:00
|
|
|
get().detachQuery(true);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
CurrentThread::QueryScope::QueryScope(Context & query_context)
|
|
|
|
{
|
|
|
|
CurrentThread::initializeQuery();
|
|
|
|
CurrentThread::attachQueryContext(query_context);
|
|
|
|
}
|
|
|
|
|
2018-09-26 21:19:49 +00:00
|
|
|
void CurrentThread::QueryScope::logPeakMemoryUsage()
|
|
|
|
{
|
|
|
|
log_peak_memory_usage_in_destructor = false;
|
|
|
|
CurrentThread::getGroup()->memory_tracker.logPeakMemoryUsage();
|
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
CurrentThread::QueryScope::~QueryScope()
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2018-09-26 21:19:49 +00:00
|
|
|
if (log_peak_memory_usage_in_destructor)
|
|
|
|
logPeakMemoryUsage();
|
2018-09-26 21:10:43 +00:00
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
CurrentThread::detachQueryIfNotDetached();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException("CurrentThread", __PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|