2018-06-20 17:49:52 +00:00
|
|
|
#include <Common/ThreadStatus.h>
|
2020-01-28 11:59:06 +00:00
|
|
|
|
2021-06-18 13:44:08 +00:00
|
|
|
#include <DataStreams/PushingToViewsBlockOutputStream.h>
|
2020-01-30 14:31:01 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2020-11-09 15:07:38 +00:00
|
|
|
#include <Interpreters/OpenTelemetrySpanLog.h>
|
2021-06-18 13:44:08 +00:00
|
|
|
#include <Interpreters/ProcessList.h>
|
2020-01-30 14:31:01 +00:00
|
|
|
#include <Interpreters/QueryThreadLog.h>
|
2021-06-18 13:44:08 +00:00
|
|
|
#include <Interpreters/QueryViewsLog.h>
|
|
|
|
#include <Parsers/formatAST.h>
|
2018-06-20 17:49:52 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
|
|
|
#include <Common/Exception.h>
|
2021-06-21 15:58:15 +00:00
|
|
|
#include <Common/ProfileEvents.h>
|
2019-03-04 13:03:32 +00:00
|
|
|
#include <Common/QueryProfiler.h>
|
2021-06-21 15:58:15 +00:00
|
|
|
#include <Common/SensitiveDataMasker.h>
|
2020-01-30 14:31:01 +00:00
|
|
|
#include <Common/ThreadProfileEvents.h>
|
|
|
|
#include <Common/TraceCollector.h>
|
2020-06-07 17:29:34 +00:00
|
|
|
#include <common/errnoToString.h>
|
2020-01-30 14:31:01 +00:00
|
|
|
|
2020-01-28 11:59:06 +00:00
|
|
|
#if defined(OS_LINUX)
|
|
|
|
# include <Common/hasLinuxCapability.h>
|
2019-07-06 18:02:28 +00:00
|
|
|
|
2020-01-28 11:59:06 +00:00
|
|
|
# include <sys/time.h>
|
|
|
|
# include <sys/resource.h>
|
2019-07-06 18:02:28 +00:00
|
|
|
#endif
|
|
|
|
|
2021-06-21 15:58:15 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event SelectedRows;
|
|
|
|
extern const Event SelectedBytes;
|
|
|
|
extern const Event InsertedRows;
|
|
|
|
extern const Event InsertedBytes;
|
|
|
|
}
|
|
|
|
|
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
|
|
|
|
{
|
|
|
|
|
2019-07-06 18:02:28 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2019-07-06 18:02:28 +00:00
|
|
|
extern const int CANNOT_SET_THREAD_PRIORITY;
|
|
|
|
}
|
|
|
|
|
2020-10-16 19:33:19 +00:00
|
|
|
void ThreadStatus::applyQuerySettings()
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context_ptr = query_context.lock();
|
|
|
|
assert(query_context_ptr);
|
|
|
|
const Settings & settings = query_context_ptr->getSettingsRef();
|
2020-10-16 19:22:14 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
query_id = query_context_ptr->getCurrentQueryId();
|
2020-10-16 19:33:19 +00:00
|
|
|
initQueryProfiler();
|
|
|
|
|
2020-10-16 19:22:14 +00:00
|
|
|
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 = settings.os_thread_priority;
|
|
|
|
if (new_os_thread_priority && hasLinuxCapability(CAP_SYS_NICE))
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Setting nice to {}", new_os_thread_priority);
|
|
|
|
|
|
|
|
if (0 != setpriority(PRIO_PROCESS, thread_id, new_os_thread_priority))
|
|
|
|
throwFromErrno("Cannot 'setpriority'", ErrorCodes::CANNOT_SET_THREAD_PRIORITY);
|
|
|
|
|
|
|
|
os_thread_priority = new_os_thread_priority;
|
|
|
|
}
|
|
|
|
#endif
|
2020-10-16 19:33:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void ThreadStatus::attachQueryContext(ContextPtr query_context_)
|
2020-10-16 19:33:19 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
query_context = query_context_;
|
2020-10-16 19:22:14 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (global_context.expired())
|
|
|
|
global_context = query_context_->getGlobalContext();
|
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);
|
2020-01-28 11:59:06 +00:00
|
|
|
|
2019-02-04 15:39:08 +00:00
|
|
|
thread_group->query_context = query_context;
|
2021-04-10 23:33:54 +00:00
|
|
|
if (thread_group->global_context.expired())
|
2019-02-04 15:39:08 +00:00
|
|
|
thread_group->global_context = global_context;
|
|
|
|
}
|
2020-01-28 11:59:06 +00:00
|
|
|
|
2020-11-18 17:43:18 +00:00
|
|
|
// Generate new span for thread manually here, because we can't depend
|
|
|
|
// on OpenTelemetrySpanHolder due to link order issues.
|
|
|
|
// FIXME why and how is this different from setupState()?
|
2021-04-10 23:33:54 +00:00
|
|
|
thread_trace_context = query_context_->query_trace_context;
|
2021-05-04 22:42:14 +00:00
|
|
|
if (thread_trace_context.trace_id != UUID())
|
2020-11-18 17:43:18 +00:00
|
|
|
{
|
|
|
|
thread_trace_context.span_id = thread_local_rng();
|
|
|
|
}
|
|
|
|
|
2020-10-16 19:33:19 +00:00
|
|
|
applyQuerySettings();
|
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-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-05 14:15:05 +00:00
|
|
|
current_thread->detachQuery(true, true);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2020-01-28 12:01:45 +00:00
|
|
|
void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
|
|
|
assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__);
|
|
|
|
|
2020-01-28 12:01:45 +00:00
|
|
|
/// Attach or init current thread to thread group and copy useful information from it
|
2018-06-20 17:49:52 +00:00
|
|
|
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
|
|
|
|
2020-01-28 12:01:45 +00:00
|
|
|
/// NOTE: thread may be attached multiple times if it is reused from a thread pool.
|
2020-02-02 02:27:15 +00:00
|
|
|
thread_group->thread_ids.emplace_back(thread_id);
|
2020-01-28 12:01:45 +00:00
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
logs_queue_ptr = thread_group->logs_queue_ptr;
|
2020-07-09 04:15:45 +00:00
|
|
|
fatal_error_callback = thread_group->fatal_error_callback;
|
2018-06-20 17:49:52 +00:00
|
|
|
query_context = thread_group->query_context;
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (global_context.expired())
|
2018-06-20 17:49:52 +00:00
|
|
|
global_context = thread_group->global_context;
|
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (auto query_context_ptr = query_context.lock())
|
2020-11-09 15:07:38 +00:00
|
|
|
{
|
2020-10-16 19:33:19 +00:00
|
|
|
applyQuerySettings();
|
2019-07-06 18:02:28 +00:00
|
|
|
|
2020-11-18 17:43:18 +00:00
|
|
|
// Generate new span for thread manually here, because we can't depend
|
|
|
|
// on OpenTelemetrySpanHolder due to link order issues.
|
2021-04-10 23:33:54 +00:00
|
|
|
thread_trace_context = query_context_ptr->query_trace_context;
|
2021-05-04 22:42:14 +00:00
|
|
|
if (thread_trace_context.trace_id != UUID())
|
2020-11-10 05:50:32 +00:00
|
|
|
{
|
2020-11-18 17:43:18 +00:00
|
|
|
thread_trace_context.span_id = thread_local_rng();
|
2020-11-09 15:07:38 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-11-18 17:43:18 +00:00
|
|
|
thread_trace_context.trace_id = 0;
|
2020-11-09 15:07:38 +00:00
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
initPerformanceCounters();
|
2019-01-18 13:10:43 +00:00
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
thread_state = ThreadState::AttachedToQuery;
|
|
|
|
}
|
|
|
|
|
2020-01-28 12:01:45 +00:00
|
|
|
void ThreadStatus::initializeQuery()
|
|
|
|
{
|
|
|
|
setupState(std::make_shared<ThreadGroupStatus>());
|
|
|
|
|
|
|
|
/// No need to lock on mutex here
|
|
|
|
thread_group->memory_tracker.setDescription("(for query)");
|
2020-02-02 02:27:15 +00:00
|
|
|
thread_group->master_thread_id = thread_id;
|
2020-01-28 12:01:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!thread_group_)
|
|
|
|
throw Exception("Attempt to attach to nullptr thread group", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
setupState(thread_group_);
|
|
|
|
}
|
|
|
|
|
2020-09-15 06:29:49 +00:00
|
|
|
inline UInt64 time_in_nanoseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
|
|
|
|
{
|
|
|
|
return std::chrono::duration_cast<std::chrono::nanoseconds>(timepoint.time_since_epoch()).count();
|
|
|
|
}
|
|
|
|
|
|
|
|
inline UInt64 time_in_microseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
|
|
|
|
{
|
|
|
|
return std::chrono::duration_cast<std::chrono::microseconds>(timepoint.time_since_epoch()).count();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
inline UInt64 time_in_seconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
|
|
|
|
{
|
|
|
|
return std::chrono::duration_cast<std::chrono::seconds>(timepoint.time_since_epoch()).count();
|
|
|
|
}
|
|
|
|
|
2020-05-20 18:25:49 +00:00
|
|
|
void ThreadStatus::initPerformanceCounters()
|
|
|
|
{
|
|
|
|
performance_counters_finalized = false;
|
|
|
|
|
|
|
|
/// Clear stats from previous query if a new query is started
|
|
|
|
/// TODO: make separate query_thread_performance_counters and thread_performance_counters
|
|
|
|
performance_counters.resetCounters();
|
|
|
|
memory_tracker.resetCounters();
|
|
|
|
memory_tracker.setDescription("(for thread)");
|
|
|
|
|
2020-09-15 06:29:49 +00:00
|
|
|
// query_start_time_{microseconds, nanoseconds} are all constructed from the same time point
|
2020-10-27 11:04:03 +00:00
|
|
|
// to ensure that they are all equal up to the precision of a second.
|
2020-09-15 06:29:49 +00:00
|
|
|
const auto now = std::chrono::system_clock::now();
|
2020-09-15 02:37:44 +00:00
|
|
|
|
2020-09-15 06:29:49 +00:00
|
|
|
query_start_time_nanoseconds = time_in_nanoseconds(now);
|
|
|
|
query_start_time = time_in_seconds(now);
|
|
|
|
query_start_time_microseconds = time_in_microseconds(now);
|
2020-05-20 18:25:49 +00:00
|
|
|
++queries_started;
|
|
|
|
|
2020-10-07 18:32:36 +00:00
|
|
|
// query_start_time_nanoseconds cannot be used here since RUsageCounters expect CLOCK_MONOTONIC
|
|
|
|
*last_rusage = RUsageCounters::current();
|
2020-05-20 18:25:49 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (auto query_context_ptr = query_context.lock())
|
2020-05-20 18:25:49 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
const Settings & settings = query_context_ptr->getSettingsRef();
|
2020-05-20 18:25:49 +00:00
|
|
|
if (settings.metrics_perf_events_enabled)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2020-05-29 13:04:32 +00:00
|
|
|
current_thread_counters.initializeProfileEvents(
|
|
|
|
settings.metrics_perf_events_list);
|
2020-05-20 18:25:49 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!taskstats)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
taskstats = TasksStatsCounters::create(thread_id);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (taskstats)
|
|
|
|
taskstats->reset();
|
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
void ThreadStatus::finalizePerformanceCounters()
|
|
|
|
{
|
|
|
|
if (performance_counters_finalized)
|
|
|
|
return;
|
|
|
|
|
|
|
|
performance_counters_finalized = true;
|
|
|
|
updatePerformanceCounters();
|
|
|
|
|
2020-06-29 12:48:18 +00:00
|
|
|
// We want to close perf file descriptors if the perf events were enabled for
|
|
|
|
// one query. What this code does in practice is less clear -- e.g., if I run
|
|
|
|
// 'select 1 settings metrics_perf_events_enabled = 1', I still get
|
|
|
|
// query_context->getSettingsRef().metrics_perf_events_enabled == 0 *shrug*.
|
2020-05-20 18:25:49 +00:00
|
|
|
bool close_perf_descriptors = true;
|
2021-04-10 23:33:54 +00:00
|
|
|
if (auto query_context_ptr = query_context.lock())
|
|
|
|
close_perf_descriptors = !query_context_ptr->getSettingsRef().metrics_perf_events_enabled;
|
2020-05-20 18:25:49 +00:00
|
|
|
|
2020-02-19 16:35:01 +00:00
|
|
|
try
|
|
|
|
{
|
2020-05-29 13:04:32 +00:00
|
|
|
current_thread_counters.finalizeProfileEvents(performance_counters);
|
2020-05-20 18:25:49 +00:00
|
|
|
if (close_perf_descriptors)
|
2020-05-29 13:04:32 +00:00
|
|
|
current_thread_counters.closeEventDescriptors();
|
2020-02-19 16:35:01 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log);
|
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
try
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
auto global_context_ptr = global_context.lock();
|
|
|
|
auto query_context_ptr = query_context.lock();
|
|
|
|
if (global_context_ptr && query_context_ptr)
|
2018-09-28 03:40:09 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
const auto & settings = query_context_ptr->getSettingsRef();
|
2018-09-28 03:40:09 +00:00
|
|
|
if (settings.log_queries && settings.log_query_threads)
|
2020-10-29 19:28:46 +00:00
|
|
|
{
|
|
|
|
const auto now = std::chrono::system_clock::now();
|
|
|
|
Int64 query_duration_ms = (time_in_microseconds(now) - query_start_time_microseconds) / 1000;
|
|
|
|
if (query_duration_ms >= settings.log_queries_min_query_duration_ms.totalMilliseconds())
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
if (auto thread_log = global_context_ptr->getQueryThreadLog())
|
|
|
|
logToQueryThreadLog(*thread_log, query_context_ptr->getCurrentDatabase(), now);
|
2020-10-29 19:28:46 +00:00
|
|
|
}
|
|
|
|
}
|
2018-09-28 03:40:09 +00:00
|
|
|
}
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
void ThreadStatus::initQueryProfiler()
|
|
|
|
{
|
2019-07-06 20:29:00 +00:00
|
|
|
/// query profilers are useless without trace collector
|
2021-04-10 23:33:54 +00:00
|
|
|
auto global_context_ptr = global_context.lock();
|
|
|
|
if (!global_context_ptr || !global_context_ptr->hasTraceCollector())
|
2019-07-06 20:29:00 +00:00
|
|
|
return;
|
2019-01-18 13:10:43 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context_ptr = query_context.lock();
|
|
|
|
assert(query_context_ptr);
|
|
|
|
const auto & settings = query_context_ptr->getSettingsRef();
|
2019-01-18 13:10:43 +00:00
|
|
|
|
2020-01-26 11:19:04 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
if (settings.query_profiler_real_time_period_ns > 0)
|
2020-02-02 02:27:15 +00:00
|
|
|
query_profiler_real = std::make_unique<QueryProfilerReal>(thread_id,
|
2020-01-26 11:19:04 +00:00
|
|
|
/* period */ static_cast<UInt32>(settings.query_profiler_real_time_period_ns));
|
|
|
|
|
|
|
|
if (settings.query_profiler_cpu_time_period_ns > 0)
|
2020-02-02 02:27:15 +00:00
|
|
|
query_profiler_cpu = std::make_unique<QueryProfilerCpu>(thread_id,
|
2020-01-26 11:19:04 +00:00
|
|
|
/* period */ static_cast<UInt32>(settings.query_profiler_cpu_time_period_ns));
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// QueryProfiler is optional.
|
|
|
|
tryLogCurrentException("ThreadStatus", "Cannot initialize QueryProfiler");
|
|
|
|
}
|
2018-12-25 00:19:17 +00:00
|
|
|
}
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
void ThreadStatus::finalizeQueryProfiler()
|
|
|
|
{
|
2019-07-06 20:29:00 +00:00
|
|
|
query_profiler_real.reset();
|
|
|
|
query_profiler_cpu.reset();
|
2018-12-25 00:19:17 +00:00
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
|
|
|
|
{
|
2021-04-14 20:37:08 +00:00
|
|
|
MemoryTracker::LockExceptionInThread lock(VariableContext::Global);
|
Lock MEMORY_LIMIT_EXCEEDED in ThreadStatus::detachQuery()
Found with fuzzer [1]:
<Fatal> BaseDaemon: (version 21.5.1.6440, build id: 3B097C902DDAA35688D90750552ED499DC5D10A0) (from thread 8012) Terminate called for uncaught exception:
Code: 241, e.displayText() = DB::Exception: Memory limit (for user) exceeded: would use 153.51 MiB (attempt to allocate chunk of 4194368 bytes), maximum: 150.00 MiB, Stack trace (when copying this message, always include the lines below):
0. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/exception:133: Poco::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int) @ 0x26fdcdd9 in /usr/bin/clickhouse
1. ./obj-x86_64-linux-gnu/../src/Common/Exception.cpp:57: DB::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int, bool) @ 0xad0df02 in /usr/bin/clickhouse
2. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/string:1444: DB::Exception::Exception<char const*, char const*, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, long&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >(int, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, char const*&&, char const*&&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >&&, long&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >&&) @ 0xacc7bef in /usr/bin/clickhouse
3. ./obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:219: MemoryTracker::alloc(long) @ 0xacc65eb in /usr/bin/clickhouse
4. ./obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:0: MemoryTracker::alloc(long) @ 0xacc5dad in /usr/bin/clickhouse
5. ./obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:0: MemoryTracker::alloc(long) @ 0xacc5dad in /usr/bin/clickhouse
6. ./obj-x86_64-linux-gnu/../src/Common/AllocatorWithMemoryTracking.h:35: AllocatorWithMemoryTracking<DB::Field>::allocate(unsigned long) @ 0xad0a2fe in /usr/bin/clickhouse
7. void std::__1::vector<DB::Field, AllocatorWithMemoryTracking<DB::Field> >::__push_back_slow_path<DB::Field>(DB::Field&&) @ 0x11712a51 in /usr/bin/clickhouse
8. ./obj-x86_64-linux-gnu/../src/Interpreters/ThreadStatusExt.cpp:356: DB::ThreadStatus::detachQuery(bool, bool) @ 0x1f5d5237 in /usr/bin/clickhouse
9. ./obj-x86_64-linux-gnu/../src/Processors/Executors/PipelineExecutor.cpp:0: void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<DB::PipelineExecutor::executeImpl(unsigned long)::$_4>(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) @ 0x20c488e6 in /usr/bin/clickhouse
10. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:0: ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) @ 0xad9f6cc in /usr/bin/clickhouse
11. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1655: void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) @ 0xada8264 in /usr/bin/clickhouse
12. start_thread @ 0x9609 in /usr/lib/x86_64-linux-gnu/libpthread-2.31.so
13. __clone @ 0x122293 in /usr/lib/x86_64-linux-gnu/libc-2.31.so
(version 21.5.1.6440)
[1]: https://clickhouse-test-reports.s3.yandex.net/22583/69296876005c0fa171c755f8b224e4d58192c402/stress_test_(address).html#fail1
2021-04-04 09:23:04 +00:00
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
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
|
|
|
|
2020-11-10 05:50:32 +00:00
|
|
|
std::shared_ptr<OpenTelemetrySpanLog> opentelemetry_span_log;
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context_ptr = query_context.lock();
|
2021-05-04 22:42:14 +00:00
|
|
|
if (thread_trace_context.trace_id != UUID() && query_context_ptr)
|
2020-11-10 05:50:32 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
opentelemetry_span_log = query_context_ptr->getOpenTelemetrySpanLog();
|
2020-11-10 05:50:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (opentelemetry_span_log)
|
2020-11-09 15:07:38 +00:00
|
|
|
{
|
2020-11-10 05:50:32 +00:00
|
|
|
// Log the current thread span.
|
|
|
|
// We do this manually, because we can't use OpenTelemetrySpanHolder as a
|
|
|
|
// ThreadStatus member, because of linking issues. This file is linked
|
|
|
|
// separately, so we can reference OpenTelemetrySpanLog here, but if we had
|
|
|
|
// the span holder as a field, we would have to reference it in the
|
|
|
|
// destructor, which is in another library.
|
|
|
|
OpenTelemetrySpanLogElement span;
|
|
|
|
|
2020-11-18 17:43:18 +00:00
|
|
|
span.trace_id = thread_trace_context.trace_id;
|
2020-11-19 15:52:11 +00:00
|
|
|
// All child span holders should be finished by the time we detach this
|
|
|
|
// thread, so the current span id should be the thread span id. If not,
|
|
|
|
// an assertion for a proper parent span in ~OpenTelemetrySpanHolder()
|
|
|
|
// is going to fail, because we're going to reset it to zero later in
|
|
|
|
// this function.
|
2020-11-18 17:43:18 +00:00
|
|
|
span.span_id = thread_trace_context.span_id;
|
2021-04-10 23:33:54 +00:00
|
|
|
assert(query_context_ptr);
|
|
|
|
span.parent_span_id = query_context_ptr->query_trace_context.span_id;
|
2020-11-10 05:50:32 +00:00
|
|
|
span.operation_name = getThreadName();
|
|
|
|
span.start_time_us = query_start_time_microseconds;
|
|
|
|
span.finish_time_us =
|
|
|
|
std::chrono::duration_cast<std::chrono::microseconds>(
|
|
|
|
std::chrono::system_clock::now().time_since_epoch()).count();
|
|
|
|
span.attribute_names.push_back("clickhouse.thread_id");
|
|
|
|
span.attribute_values.push_back(thread_id);
|
|
|
|
|
|
|
|
opentelemetry_span_log->add(span);
|
2020-11-09 15:07:38 +00:00
|
|
|
}
|
|
|
|
|
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
|
|
|
|
2020-10-24 18:34:54 +00:00
|
|
|
/// Must reset pointer to thread_group's memory_tracker, because it will be destroyed two lines below (will reset to its parent).
|
2020-10-22 20:00:51 +00:00
|
|
|
memory_tracker.setParent(thread_group->memory_tracker.getParent());
|
2018-10-05 19:29:49 +00:00
|
|
|
|
2019-12-22 20:50:33 +00:00
|
|
|
query_id.clear();
|
2021-04-10 23:33:54 +00:00
|
|
|
query_context.reset();
|
2020-11-18 17:43:18 +00:00
|
|
|
thread_trace_context.trace_id = 0;
|
|
|
|
thread_trace_context.span_id = 0;
|
2018-06-20 17:49:52 +00:00
|
|
|
thread_group.reset();
|
|
|
|
|
|
|
|
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;
|
2019-07-06 18:02:28 +00:00
|
|
|
|
|
|
|
#if defined(__linux__)
|
|
|
|
if (os_thread_priority)
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Resetting nice");
|
|
|
|
|
2020-02-02 02:27:15 +00:00
|
|
|
if (0 != setpriority(PRIO_PROCESS, thread_id, 0))
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_ERROR(log, "Cannot 'setpriority' back to zero: {}", errnoToString(ErrorCodes::CANNOT_SET_THREAD_PRIORITY, errno));
|
2019-07-06 18:02:28 +00:00
|
|
|
|
|
|
|
os_thread_priority = 0;
|
|
|
|
}
|
|
|
|
#endif
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2020-10-29 19:28:46 +00:00
|
|
|
void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point<std::chrono::system_clock> now)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
|
|
|
QueryThreadLogElement elem;
|
|
|
|
|
2020-09-15 06:49:02 +00:00
|
|
|
// construct current_time and current_time_microseconds using the same time point
|
|
|
|
// so that the two times will always be equal up to a precision of a second.
|
2020-10-29 19:28:46 +00:00
|
|
|
auto current_time = time_in_seconds(now);
|
|
|
|
auto current_time_microseconds = time_in_microseconds(now);
|
2020-09-15 01:40:06 +00:00
|
|
|
|
2020-09-15 06:49:02 +00:00
|
|
|
elem.event_time = current_time;
|
|
|
|
elem.event_time_microseconds = current_time_microseconds;
|
2018-06-20 17:49:52 +00:00
|
|
|
elem.query_start_time = query_start_time;
|
2020-08-18 21:41:01 +00:00
|
|
|
elem.query_start_time_microseconds = query_start_time_microseconds;
|
2020-10-31 08:15:27 +00:00
|
|
|
elem.query_duration_ms = (time_in_nanoseconds(now) - query_start_time_nanoseconds) / 1000000U;
|
2018-06-20 17:49:52 +00:00
|
|
|
|
2019-05-20 11:37:41 +00:00
|
|
|
elem.read_rows = progress_in.read_rows.load(std::memory_order_relaxed);
|
|
|
|
elem.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed);
|
2019-05-21 04:06:36 +00:00
|
|
|
|
2019-05-20 11:37:41 +00:00
|
|
|
/// TODO: Use written_rows and written_bytes when run time progress is implemented
|
|
|
|
elem.written_rows = progress_out.read_rows.load(std::memory_order_relaxed);
|
|
|
|
elem.written_bytes = progress_out.read_bytes.load(std::memory_order_relaxed);
|
2018-06-20 17:49:52 +00:00
|
|
|
elem.memory_usage = memory_tracker.get();
|
|
|
|
elem.peak_memory_usage = memory_tracker.getPeak();
|
|
|
|
|
|
|
|
elem.thread_name = getThreadName();
|
2020-02-02 02:27:15 +00:00
|
|
|
elem.thread_id = thread_id;
|
2018-06-20 17:49:52 +00:00
|
|
|
|
2020-10-30 18:16:10 +00:00
|
|
|
elem.current_database = current_database;
|
2018-06-20 17:49:52 +00:00
|
|
|
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
|
|
|
|
2020-02-02 02:27:15 +00:00
|
|
|
elem.master_thread_id = thread_group->master_thread_id;
|
2018-06-20 17:49:52 +00:00
|
|
|
elem.query = thread_group->query;
|
2020-12-02 12:08:03 +00:00
|
|
|
elem.normalized_query_hash = thread_group->normalized_query_hash;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context_ptr = query_context.lock();
|
|
|
|
if (query_context_ptr)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
elem.client_info = query_context_ptr->getClientInfo();
|
2018-06-20 17:49:52 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (query_context_ptr->getSettingsRef().log_profile_events != 0)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
|
|
|
/// 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);
|
|
|
|
}
|
|
|
|
|
2021-06-21 15:58:15 +00:00
|
|
|
static String getCleanQueryAst(const ASTPtr q, ContextPtr context)
|
2021-06-18 13:44:08 +00:00
|
|
|
{
|
2021-06-21 15:58:15 +00:00
|
|
|
String res = serializeAST(*q, true);
|
|
|
|
if (auto * masker = SensitiveDataMasker::getInstance())
|
|
|
|
masker->wipeSensitiveData(res);
|
|
|
|
|
|
|
|
res = res.substr(0, context->getSettingsRef().log_queries_cut_to_length);
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadStatus::logToQueryViewsLog(const ViewInfo & vinfo)
|
|
|
|
{
|
|
|
|
auto query_context_ptr = query_context.lock();
|
|
|
|
if (!query_context_ptr)
|
|
|
|
return;
|
|
|
|
auto views_log = query_context_ptr->getQueryViewsLog();
|
|
|
|
if (!views_log)
|
|
|
|
return;
|
|
|
|
|
2021-06-18 13:44:08 +00:00
|
|
|
QueryViewsLogElement element;
|
|
|
|
|
2021-06-18 16:25:19 +00:00
|
|
|
element.event_time = time_in_seconds(vinfo.runtime_stats.event_time);
|
|
|
|
element.event_time_microseconds = time_in_microseconds(vinfo.runtime_stats.event_time);
|
2021-06-18 13:44:08 +00:00
|
|
|
element.view_duration_ms = vinfo.runtime_stats.elapsed_ms;
|
|
|
|
|
2021-06-23 14:14:30 +00:00
|
|
|
element.initial_query_id = query_id;
|
2021-06-24 14:39:08 +00:00
|
|
|
element.view_name = vinfo.table_id.getFullTableName();
|
2021-06-18 13:44:08 +00:00
|
|
|
element.view_uuid = vinfo.table_id.uuid;
|
|
|
|
element.view_type = vinfo.runtime_stats.type;
|
|
|
|
if (vinfo.query)
|
2021-06-21 15:58:15 +00:00
|
|
|
element.view_query = getCleanQueryAst(vinfo.query, query_context_ptr);
|
2021-06-18 13:44:08 +00:00
|
|
|
element.view_target = vinfo.runtime_stats.target_name;
|
|
|
|
|
2021-06-21 15:58:15 +00:00
|
|
|
auto events = std::make_shared<ProfileEvents::Counters>(performance_counters.getPartiallyAtomicSnapshot());
|
|
|
|
element.read_rows = (*events)[ProfileEvents::SelectedRows];
|
|
|
|
element.read_bytes = (*events)[ProfileEvents::SelectedBytes];
|
|
|
|
element.written_rows = (*events)[ProfileEvents::InsertedRows];
|
|
|
|
element.written_bytes = (*events)[ProfileEvents::InsertedBytes];
|
|
|
|
element.peak_memory_usage = memory_tracker.getPeak() > 0 ? memory_tracker.getPeak() : 0;
|
|
|
|
if (query_context_ptr->getSettingsRef().log_profile_events != 0)
|
2021-06-18 13:44:08 +00:00
|
|
|
{
|
2021-06-21 15:58:15 +00:00
|
|
|
element.profile_counters = events;
|
2021-06-18 13:44:08 +00:00
|
|
|
}
|
|
|
|
|
2021-06-22 11:39:53 +00:00
|
|
|
element.status = vinfo.runtime_stats.event_status;
|
2021-06-18 13:44:08 +00:00
|
|
|
element.exception_code = 0;
|
|
|
|
if (vinfo.exception)
|
|
|
|
{
|
2021-06-18 16:25:19 +00:00
|
|
|
element.exception_code = getExceptionErrorCode(vinfo.exception);
|
|
|
|
element.exception = getExceptionMessage(vinfo.exception, false);
|
2021-06-21 15:58:15 +00:00
|
|
|
if (query_context_ptr->getSettingsRef().calculate_text_stack_trace)
|
|
|
|
element.stack_trace = getExceptionStackTraceString(vinfo.exception);
|
2021-06-18 13:44:08 +00:00
|
|
|
}
|
|
|
|
|
2021-06-21 15:58:15 +00:00
|
|
|
views_log->add(element);
|
2021-06-18 13:44:08 +00:00
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
void CurrentThread::initializeQuery()
|
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-05 14:15:05 +00:00
|
|
|
current_thread->initializeQuery();
|
|
|
|
current_thread->deleter = CurrentThread::defaultThreadDeleter;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group)
|
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-05 14:15:05 +00:00
|
|
|
current_thread->attachQuery(thread_group, true);
|
|
|
|
current_thread->deleter = CurrentThread::defaultThreadDeleter;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group)
|
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-05 14:15:05 +00:00
|
|
|
current_thread->attachQuery(thread_group, false);
|
|
|
|
current_thread->deleter = CurrentThread::defaultThreadDeleter;
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void CurrentThread::attachQueryContext(ContextPtr query_context)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-15 16:32:52 +00:00
|
|
|
current_thread->attachQueryContext(query_context);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::finalizePerformanceCounters()
|
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-05 14:15:05 +00:00
|
|
|
current_thread->finalizePerformanceCounters();
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::detachQuery()
|
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-05 14:15:05 +00:00
|
|
|
current_thread->detachQuery(false);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::detachQueryIfNotDetached()
|
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2019-07-05 14:15:05 +00:00
|
|
|
current_thread->detachQuery(true);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-05-31 14:49:02 +00:00
|
|
|
CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
|
|
|
CurrentThread::initializeQuery();
|
|
|
|
CurrentThread::attachQueryContext(query_context);
|
2021-04-10 23:33:54 +00:00
|
|
|
if (!query_context->hasQueryContext())
|
|
|
|
query_context->makeQueryContext();
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2018-09-26 21:19:49 +00:00
|
|
|
void CurrentThread::QueryScope::logPeakMemoryUsage()
|
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
auto group = CurrentThread::getGroup();
|
|
|
|
if (!group)
|
|
|
|
return;
|
|
|
|
|
2018-09-26 21:19:49 +00:00
|
|
|
log_peak_memory_usage_in_destructor = false;
|
2019-03-14 18:03:35 +00:00
|
|
|
group->memory_tracker.logPeakMemoryUsage();
|
2018-09-26 21:19:49 +00:00
|
|
|
}
|
|
|
|
|
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__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|