2021-08-31 13:50:56 +00:00
|
|
|
#include <mutex>
|
2018-06-20 17:49:52 +00:00
|
|
|
#include <Common/ThreadStatus.h>
|
2020-01-28 11:59:06 +00:00
|
|
|
|
2021-10-08 14:03:54 +00:00
|
|
|
#include <Processors/Transforms/buildPushingToViewsChain.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>
|
2022-01-10 19:35:42 +00:00
|
|
|
#include <Interpreters/TraceCollector.h>
|
2021-06-18 13:44:08 +00:00
|
|
|
#include <Parsers/formatAST.h>
|
2023-03-15 21:12:29 +00:00
|
|
|
#include <Parsers/queryNormalization.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>
|
2022-01-10 19:01:41 +00:00
|
|
|
#include <Common/setThreadName.h>
|
2022-06-16 17:41:32 +00:00
|
|
|
#include <Common/noexcept_scope.h>
|
2022-10-03 18:52:14 +00:00
|
|
|
#include <Common/DateLUT.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/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
|
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
ThreadGroupStatus::ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_)
|
|
|
|
: master_thread_id(CurrentThread::get().thread_id)
|
|
|
|
, query_context(query_context_)
|
|
|
|
, global_context(query_context_->getGlobalContext())
|
|
|
|
, fatal_error_callback(fatal_error_callback_)
|
|
|
|
{}
|
|
|
|
|
|
|
|
std::vector<UInt64> ThreadGroupStatus::getInvolvedThreadIds() const
|
2023-03-14 11:36:26 +00:00
|
|
|
{
|
|
|
|
std::vector<UInt64> res;
|
|
|
|
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
res.assign(thread_ids.begin(), thread_ids.end());
|
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadGroupStatus::linkThread(UInt64 thread_it)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
thread_ids.insert(thread_it);
|
|
|
|
}
|
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
ThreadGroupStatusPtr ThreadGroupStatus::createForQuery(ContextPtr query_context_, std::function<void()> fatal_error_callback_)
|
|
|
|
{
|
2023-03-15 21:12:29 +00:00
|
|
|
auto group = std::make_shared<ThreadGroupStatus>(query_context_, std::move(fatal_error_callback_));
|
2023-03-14 11:36:26 +00:00
|
|
|
group->memory_tracker.setDescription("(for query)");
|
2023-03-15 21:12:29 +00:00
|
|
|
return group;
|
|
|
|
}
|
2023-03-14 11:36:26 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadGroupStatus::attachQueryForLog(const String & query_, UInt64 normalized_hash)
|
|
|
|
{
|
|
|
|
auto hash = normalized_hash ? normalized_hash : normalizedQueryHash<false>(query_);
|
2023-03-14 11:36:26 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
shared_data.query_for_logs = query_;
|
|
|
|
shared_data.normalized_query_hash = hash;
|
|
|
|
}
|
2023-03-14 11:36:26 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadStatus::attachQueryForLog(const String & query_)
|
|
|
|
{
|
2023-03-17 10:59:44 +00:00
|
|
|
local_data.query_for_logs = query_;
|
|
|
|
local_data.normalized_query_hash = normalizedQueryHash<false>(query_);
|
2023-03-15 21:12:29 +00:00
|
|
|
|
|
|
|
if (!thread_group)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id);
|
|
|
|
|
2023-03-17 10:59:44 +00:00
|
|
|
thread_group->attachQueryForLog(local_data.query_for_logs, local_data.normalized_query_hash);
|
2023-03-15 21:12:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadGroupStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
shared_data.profile_queue_ptr = profile_queue;
|
|
|
|
}
|
|
|
|
|
|
|
|
void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue)
|
|
|
|
{
|
|
|
|
if (!thread_group)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id);
|
|
|
|
|
2023-03-17 10:59:44 +00:00
|
|
|
local_data.profile_queue_ptr = profile_queue;
|
2023-03-15 21:12:29 +00:00
|
|
|
thread_group->attachInternalProfileEventsQueue(profile_queue);
|
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue)
|
|
|
|
{
|
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
|
|
|
current_thread->attachInternalProfileEventsQueue(queue);
|
|
|
|
}
|
|
|
|
|
|
|
|
void CurrentThread::attachQueryForLog(const String & query_)
|
|
|
|
{
|
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
|
|
|
current_thread->attachQueryForLog(query_);
|
2023-03-14 11:36:26 +00:00
|
|
|
}
|
|
|
|
|
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();
|
2023-03-15 21:12:29 +00:00
|
|
|
if (!query_context_ptr)
|
|
|
|
return;
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
const Settings & settings = query_context_ptr->getSettingsRef();
|
2020-10-16 19:22:14 +00:00
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
query_id_from_query_context = 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;
|
2022-04-18 08:18:31 +00:00
|
|
|
if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast<UInt64>(untracked_memory_limit))
|
2020-10-16 19:22:14 +00:00
|
|
|
untracked_memory_limit = settings.memory_profiler_step;
|
|
|
|
|
|
|
|
#if defined(OS_LINUX)
|
|
|
|
/// Set "nice" value if required.
|
2022-10-07 10:46:45 +00:00
|
|
|
Int32 new_os_thread_priority = static_cast<Int32>(settings.os_thread_priority);
|
2020-10-16 19:22:14 +00:00
|
|
|
if (new_os_thread_priority && hasLinuxCapability(CAP_SYS_NICE))
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Setting nice to {}", new_os_thread_priority);
|
|
|
|
|
2022-10-07 10:46:45 +00:00
|
|
|
if (0 != setpriority(PRIO_PROCESS, static_cast<unsigned>(thread_id), new_os_thread_priority))
|
2020-10-16 19:22:14 +00:00
|
|
|
throwFromErrno("Cannot 'setpriority'", ErrorCodes::CANNOT_SET_THREAD_PRIORITY);
|
|
|
|
|
|
|
|
os_thread_priority = new_os_thread_priority;
|
|
|
|
}
|
|
|
|
#endif
|
2020-10-16 19:33:19 +00:00
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadStatus::attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
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_;
|
2023-03-15 21:12:29 +00:00
|
|
|
thread_group->linkThread(thread_id);
|
2018-06-20 17:49:52 +00:00
|
|
|
|
|
|
|
performance_counters.setParent(&thread_group->performance_counters);
|
|
|
|
memory_tracker.setParent(&thread_group->memory_tracker);
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
query_context = thread_group->query_context;
|
|
|
|
global_context = thread_group->global_context;
|
2023-03-14 11:36:26 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
fatal_error_callback = thread_group->fatal_error_callback;
|
2023-03-14 11:36:26 +00:00
|
|
|
|
2023-03-17 10:59:44 +00:00
|
|
|
local_data = thread_group->getSharedData();
|
2020-11-09 15:07:38 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
applyQuerySettings();
|
2018-06-20 17:49:52 +00:00
|
|
|
initPerformanceCounters();
|
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadStatus::detachFromGroup()
|
2023-03-14 11:36:26 +00:00
|
|
|
{
|
2023-03-15 21:12:29 +00:00
|
|
|
if (!thread_group)
|
|
|
|
return;
|
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
/// flash untracked memory before resetting memory_tracker parent
|
|
|
|
flushUntrackedMemory();
|
2023-03-14 11:36:26 +00:00
|
|
|
|
|
|
|
finalizeQueryProfiler();
|
|
|
|
finalizePerformanceCounters();
|
|
|
|
|
|
|
|
performance_counters.setParent(&ProfileEvents::global_counters);
|
|
|
|
|
|
|
|
memory_tracker.reset();
|
|
|
|
memory_tracker.setParent(thread_group->memory_tracker.getParent());
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
thread_group.reset();
|
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
query_id_from_query_context.clear();
|
|
|
|
query_context.reset();
|
|
|
|
|
2023-03-17 10:59:44 +00:00
|
|
|
local_data = {};
|
2023-03-14 11:36:26 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
fatal_error_callback = {};
|
2023-03-14 11:36:26 +00:00
|
|
|
|
|
|
|
#if defined(OS_LINUX)
|
|
|
|
if (os_thread_priority)
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Resetting nice");
|
|
|
|
|
|
|
|
if (0 != setpriority(PRIO_PROCESS, static_cast<int>(thread_id), 0))
|
|
|
|
LOG_ERROR(log, "Cannot 'setpriority' back to zero: {}", errnoToString());
|
|
|
|
|
|
|
|
os_thread_priority = 0;
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
}
|
|
|
|
|
2022-10-31 11:44:28 +00:00
|
|
|
void ThreadStatus::setInternalThread()
|
|
|
|
{
|
|
|
|
chassert(!query_profiler_real && !query_profiler_cpu);
|
|
|
|
internal_thread = true;
|
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadStatus::attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached)
|
2020-01-28 12:01:45 +00:00
|
|
|
{
|
2023-03-15 21:12:29 +00:00
|
|
|
if (thread_group && check_detached)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached");
|
2020-01-28 12:01:45 +00:00
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
if (!thread_group_)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to attach to nullptr thread group");
|
2020-01-28 12:01:45 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
if (thread_group)
|
2020-01-28 12:01:45 +00:00
|
|
|
return;
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
deleter = [this] () { detachFromGroup(); };
|
|
|
|
attachToGroupImpl(thread_group_);
|
2020-01-28 12:01:45 +00:00
|
|
|
}
|
|
|
|
|
2023-02-08 11:12:59 +00:00
|
|
|
ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope)
|
2023-01-23 12:45:28 +00:00
|
|
|
{
|
2023-02-08 11:12:59 +00:00
|
|
|
ProfileEvents::Counters * prev_counters = current_performance_counters;
|
|
|
|
|
2023-02-02 09:46:01 +00:00
|
|
|
if (current_performance_counters == performance_counters_scope)
|
|
|
|
/// Allow to attach the same scope multiple times
|
2023-02-08 11:12:59 +00:00
|
|
|
return prev_counters;
|
2023-02-02 09:46:01 +00:00
|
|
|
|
2023-02-20 12:28:44 +00:00
|
|
|
/// Avoid cycles when exiting local scope and attaching back to current thread counters
|
|
|
|
if (performance_counters_scope != &performance_counters)
|
2023-02-15 12:18:17 +00:00
|
|
|
performance_counters_scope->setParent(&performance_counters);
|
2023-02-20 12:28:44 +00:00
|
|
|
|
2023-01-31 12:34:07 +00:00
|
|
|
current_performance_counters = performance_counters_scope;
|
|
|
|
|
2023-02-08 11:12:59 +00:00
|
|
|
return prev_counters;
|
2023-01-23 12:45:28 +00:00
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadStatus::TimePoint::setUp()
|
2023-03-14 11:36:26 +00:00
|
|
|
{
|
2023-03-15 21:12:29 +00:00
|
|
|
point = std::chrono::system_clock::now();
|
2023-03-14 11:36:26 +00:00
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
UInt64 ThreadStatus::TimePoint::nanoseconds() const
|
|
|
|
{
|
|
|
|
return timeInNanoseconds(point);
|
|
|
|
}
|
|
|
|
|
|
|
|
UInt64 ThreadStatus::TimePoint::microseconds() const
|
2023-03-14 11:36:26 +00:00
|
|
|
{
|
2023-03-15 21:12:29 +00:00
|
|
|
return timeInMicroseconds(point);
|
|
|
|
}
|
|
|
|
|
|
|
|
UInt64 ThreadStatus::TimePoint::seconds() const
|
|
|
|
{
|
|
|
|
return timeInSeconds(point);
|
2023-03-14 11:36:26 +00:00
|
|
|
}
|
|
|
|
|
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)");
|
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
query_start_time.setUp();
|
2020-05-20 18:25:49 +00:00
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
// query_start_time.nanoseconds cannot be used here since RUsageCounters expect CLOCK_MONOTONIC
|
2020-10-07 18:32:36 +00:00
|
|
|
*last_rusage = RUsageCounters::current();
|
2020-05-20 18:25:49 +00:00
|
|
|
|
2022-10-31 11:44:28 +00:00
|
|
|
if (!internal_thread)
|
2020-05-20 18:25:49 +00:00
|
|
|
{
|
2022-10-31 11:44:28 +00:00
|
|
|
if (auto query_context_ptr = query_context.lock())
|
|
|
|
{
|
|
|
|
const Settings & settings = query_context_ptr->getSettingsRef();
|
|
|
|
if (settings.metrics_perf_events_enabled)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
current_thread_counters.initializeProfileEvents(
|
|
|
|
settings.metrics_perf_events_list);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!taskstats)
|
2020-05-20 18:25:49 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2022-10-31 11:44:28 +00:00
|
|
|
taskstats = TasksStatsCounters::create(thread_id);
|
2020-05-20 18:25:49 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2022-10-31 11:44:28 +00:00
|
|
|
tryLogCurrentException(log);
|
2020-05-20 18:25:49 +00:00
|
|
|
}
|
|
|
|
}
|
2022-10-31 11:44:28 +00:00
|
|
|
if (taskstats)
|
|
|
|
taskstats->reset();
|
2020-05-20 18:25:49 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-20 17:49:52 +00:00
|
|
|
void ThreadStatus::finalizePerformanceCounters()
|
|
|
|
{
|
2022-10-31 11:44:28 +00:00
|
|
|
if (performance_counters_finalized || internal_thread)
|
2018-06-20 17:49:52 +00:00
|
|
|
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();
|
2023-03-15 21:12:29 +00:00
|
|
|
Int64 query_duration_ms = std::chrono::duration_cast<std::chrono::microseconds>(now - query_start_time.point).count();
|
2020-10-29 19:28:46 +00:00
|
|
|
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())
|
2023-03-15 21:12:29 +00:00
|
|
|
logToQueryThreadLog(*thread_log, query_context_ptr->getCurrentDatabase());
|
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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-08-09 17:38:29 +00:00
|
|
|
void ThreadStatus::resetPerformanceCountersLastUsage()
|
|
|
|
{
|
|
|
|
*last_rusage = RUsageCounters::current();
|
|
|
|
if (taskstats)
|
|
|
|
taskstats->reset();
|
|
|
|
}
|
|
|
|
|
2019-02-09 22:40:47 +00:00
|
|
|
void ThreadStatus::initQueryProfiler()
|
|
|
|
{
|
2022-10-31 11:44:28 +00:00
|
|
|
if (internal_thread)
|
2021-07-01 11:27:11 +00:00
|
|
|
return;
|
|
|
|
|
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,
|
2021-11-25 18:06:57 +00:00
|
|
|
/* period= */ static_cast<UInt32>(settings.query_profiler_real_time_period_ns));
|
2020-01-26 11:19:04 +00:00
|
|
|
|
|
|
|
if (settings.query_profiler_cpu_time_period_ns > 0)
|
2021-08-25 00:58:49 +00:00
|
|
|
query_profiler_cpu = std::make_unique<QueryProfilerCPU>(thread_id,
|
2021-11-25 18:06:57 +00:00
|
|
|
/* period= */ static_cast<UInt32>(settings.query_profiler_cpu_time_period_ns));
|
2020-01-26 11:19:04 +00:00
|
|
|
}
|
|
|
|
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
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database)
|
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.
|
2023-03-14 11:36:26 +00:00
|
|
|
TimePoint current_time;
|
2023-03-15 21:12:29 +00:00
|
|
|
current_time.setUp();
|
2020-09-15 01:40:06 +00:00
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
elem.event_time = current_time.seconds();
|
|
|
|
elem.event_time_microseconds = current_time.microseconds();
|
|
|
|
elem.query_start_time = query_start_time.seconds();
|
|
|
|
elem.query_start_time_microseconds = query_start_time.microseconds();
|
|
|
|
elem.query_duration_ms = std::chrono::duration_cast<std::chrono::milliseconds>(current_time.point - query_start_time.point).count();
|
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
|
|
|
|
2021-12-09 13:00:12 +00:00
|
|
|
elem.written_rows = progress_out.written_rows.load(std::memory_order_relaxed);
|
|
|
|
elem.written_bytes = progress_out.written_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)
|
|
|
|
{
|
2023-03-15 21:12:29 +00:00
|
|
|
elem.master_thread_id = thread_group->master_thread_id;
|
2023-03-17 10:59:44 +00:00
|
|
|
elem.query = local_data.query_for_logs;
|
|
|
|
elem.normalized_query_hash = local_data.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()
|
2021-10-11 15:56:23 +00:00
|
|
|
elem.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(performance_counters.getPartiallyAtomicSnapshot());
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2021-08-09 17:38:29 +00:00
|
|
|
void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo)
|
2021-06-21 15:58:15 +00:00
|
|
|
{
|
|
|
|
auto query_context_ptr = query_context.lock();
|
|
|
|
if (!query_context_ptr)
|
|
|
|
return;
|
2023-03-14 11:36:26 +00:00
|
|
|
|
2021-06-21 15:58:15 +00:00
|
|
|
auto views_log = query_context_ptr->getQueryViewsLog();
|
|
|
|
if (!views_log)
|
|
|
|
return;
|
|
|
|
|
2021-06-18 13:44:08 +00:00
|
|
|
QueryViewsLogElement element;
|
|
|
|
|
2022-10-03 18:52:14 +00:00
|
|
|
element.event_time = timeInSeconds(vinfo.runtime_stats->event_time);
|
|
|
|
element.event_time_microseconds = timeInMicroseconds(vinfo.runtime_stats->event_time);
|
2021-09-21 10:35:41 +00:00
|
|
|
element.view_duration_ms = vinfo.runtime_stats->elapsed_ms;
|
2021-06-18 13:44:08 +00:00
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
element.initial_query_id = query_id_from_query_context;
|
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;
|
2021-09-21 10:35:41 +00:00
|
|
|
element.view_type = vinfo.runtime_stats->type;
|
2021-06-18 13:44:08 +00:00
|
|
|
if (vinfo.query)
|
2021-06-21 15:58:15 +00:00
|
|
|
element.view_query = getCleanQueryAst(vinfo.query, query_context_ptr);
|
2021-09-21 10:35:41 +00:00
|
|
|
element.view_target = vinfo.runtime_stats->target_name;
|
2021-06-18 13:44:08 +00:00
|
|
|
|
2021-07-01 15:19:30 +00:00
|
|
|
element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed);
|
|
|
|
element.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed);
|
2021-12-09 13:00:12 +00:00
|
|
|
element.written_rows = progress_out.written_rows.load(std::memory_order_relaxed);
|
|
|
|
element.written_bytes = progress_out.written_bytes.load(std::memory_order_relaxed);
|
2021-06-21 15:58:15 +00:00
|
|
|
element.peak_memory_usage = memory_tracker.getPeak() > 0 ? memory_tracker.getPeak() : 0;
|
|
|
|
if (query_context_ptr->getSettingsRef().log_profile_events != 0)
|
2023-03-14 11:36:26 +00:00
|
|
|
element.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(
|
|
|
|
performance_counters.getPartiallyAtomicSnapshot());
|
2021-06-18 13:44:08 +00:00
|
|
|
|
2021-09-21 10:35:41 +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
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void CurrentThread::attachToGroup(const ThreadGroupStatusPtr & thread_group)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2023-03-15 21:12:29 +00:00
|
|
|
current_thread->attachToGroup(thread_group, true);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void CurrentThread::attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2023-03-15 21:12:29 +00:00
|
|
|
current_thread->attachToGroup(thread_group, false);
|
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
|
|
|
}
|
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
void CurrentThread::detachFromGroupIfNotDetached()
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2019-03-14 18:03:35 +00:00
|
|
|
if (unlikely(!current_thread))
|
|
|
|
return;
|
2023-03-15 21:12:29 +00:00
|
|
|
current_thread->detachFromGroup();
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2023-03-03 00:22:04 +00:00
|
|
|
CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::function<void()> fatal_error_callback)
|
2018-06-20 17:49:52 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
if (!query_context->hasQueryContext())
|
|
|
|
query_context->makeQueryContext();
|
2023-03-14 11:36:26 +00:00
|
|
|
|
|
|
|
auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback));
|
2023-03-17 09:49:41 +00:00
|
|
|
CurrentThread::attachToGroup(group);
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
|
2023-03-03 00:22:04 +00:00
|
|
|
CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function<void()> fatal_error_callback)
|
2022-04-01 14:45:15 +00:00
|
|
|
{
|
|
|
|
if (!query_context->hasQueryContext())
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::LOGICAL_ERROR, "Cannot initialize query scope without query context");
|
|
|
|
|
2023-03-14 11:36:26 +00:00
|
|
|
auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback));
|
2023-03-17 09:49:41 +00:00
|
|
|
CurrentThread::attachToGroup(group);
|
2022-04-01 14:45:15 +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
|
|
|
|
2023-03-15 21:12:29 +00:00
|
|
|
CurrentThread::detachFromGroupIfNotDetached();
|
2018-06-20 17:49:52 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException("CurrentThread", __PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|