ClickHouse/src/Common/ThreadStatus.h

326 lines
10 KiB
C++
Raw Normal View History

#pragma once
#include <Core/SettingsEnums.h>
#include <Interpreters/Context_fwd.h>
#include <IO/Progress.h>
#include <Common/MemoryTracker.h>
#include <Common/ProfileEvents.h>
2023-03-22 13:51:07 +00:00
#include <Common/Stopwatch.h>
2021-10-02 07:13:14 +00:00
#include <base/StringRef.h>
#include <boost/noncopyable.hpp>
#include <functional>
#include <map>
#include <memory>
#include <mutex>
#include <unordered_set>
namespace Poco
{
class Logger;
}
template <class T>
class ConcurrentBoundedQueue;
namespace DB
{
class QueryStatus;
class ThreadStatus;
class QueryProfilerReal;
2021-08-25 00:58:49 +00:00
class QueryProfilerCPU;
class QueryThreadLog;
class TasksStatsCounters;
struct RUsageCounters;
2020-02-19 16:35:01 +00:00
struct PerfEventsCounters;
class TaskStatsInfoGetter;
class InternalTextLogsQueue;
struct ViewRuntimeData;
2021-06-18 13:44:08 +00:00
class QueryViewsLog;
using InternalTextLogsQueuePtr = std::shared_ptr<InternalTextLogsQueue>;
using InternalTextLogsQueueWeakPtr = std::weak_ptr<InternalTextLogsQueue>;
2021-09-02 14:27:19 +00:00
using InternalProfileEventsQueue = ConcurrentBoundedQueue<Block>;
using InternalProfileEventsQueuePtr = std::shared_ptr<InternalProfileEventsQueue>;
using InternalProfileEventsQueueWeakPtr = std::weak_ptr<InternalProfileEventsQueue>;
using ThreadStatusPtr = ThreadStatus *;
2018-09-06 00:28:15 +00:00
/** Thread group is a collection of threads dedicated to single task
* (query or other process like background merge).
*
* ProfileEvents (counters) from a thread are propagated to thread group.
*
* Create via CurrentThread::initializeQuery (for queries) or directly (for various background tasks).
* Use via CurrentThread::getGroup.
*/
class ThreadGroup;
using ThreadGroupPtr = std::shared_ptr<ThreadGroup>;
class ThreadGroup
{
public:
ThreadGroup();
2023-03-15 21:12:29 +00:00
using FatalErrorCallback = std::function<void()>;
ThreadGroup(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {});
2023-03-15 21:12:29 +00:00
/// The first thread created this thread group
2023-03-15 21:12:29 +00:00
const UInt64 master_thread_id;
2023-03-15 21:12:29 +00:00
/// Set up at creation, no race when reading
const ContextWeakPtr query_context;
const ContextWeakPtr global_context;
2023-03-15 21:12:29 +00:00
const FatalErrorCallback fatal_error_callback;
2023-03-15 21:12:29 +00:00
ProfileEvents::Counters performance_counters{VariableContext::Process};
MemoryTracker memory_tracker{VariableContext::Process};
2023-03-15 21:12:29 +00:00
struct SharedData
{
InternalProfileEventsQueueWeakPtr profile_queue_ptr;
2019-07-09 10:39:05 +00:00
2023-03-15 21:12:29 +00:00
InternalTextLogsQueueWeakPtr logs_queue_ptr;
LogsLevel client_logs_level = LogsLevel::none;
2023-03-15 21:12:29 +00:00
String query_for_logs;
UInt64 normalized_query_hash = 0;
};
2023-03-15 21:12:29 +00:00
SharedData getSharedData()
{
2023-03-17 10:59:44 +00:00
/// Critical section for making the copy of shared_data
2023-03-15 21:12:29 +00:00
std::lock_guard lock(mutex);
return shared_data;
}
2023-03-15 21:12:29 +00:00
/// Mutation shared data
void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level);
void attachQueryForLog(const String & query_, UInt64 normalized_hash = 0);
void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue);
2023-03-15 21:12:29 +00:00
/// When new query starts, new thread group is created for it, current thread becomes master thread of the query
static ThreadGroupPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {});
static ThreadGroupPtr createForBackgroundProcess(ContextPtr storage_context);
2023-03-27 12:31:29 +00:00
2023-03-15 21:12:29 +00:00
std::vector<UInt64> getInvolvedThreadIds() const;
void linkThread(UInt64 thread_it);
private:
2023-03-15 21:12:29 +00:00
mutable std::mutex mutex;
/// Set up at creation, no race when reading
2023-03-15 21:12:29 +00:00
SharedData shared_data;
/// Set of all thread ids which has been attached to the group
std::unordered_set<UInt64> thread_ids;
};
/**
* Since merge is executed with multiple threads, this class
* switches the parent MemoryTracker as part of the thread group to account all the memory used.
*/
class ThreadGroupSwitcher : private boost::noncopyable
{
public:
explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group);
~ThreadGroupSwitcher();
private:
ThreadGroupPtr prev_thread_group;
};
/**
* We use **constinit** here to tell the compiler the current_thread variable is initialized.
* If we didn't help the compiler, then it would most likely add a check before every use of the variable to initialize it if needed.
* Instead it will trust that we are doing the right thing (and we do initialize it to nullptr) and emit more optimal code.
* This is noticeable in functions like CurrentMemoryTracker::free and CurrentMemoryTracker::allocImpl
* See also:
* - https://en.cppreference.com/w/cpp/language/constinit
* - https://github.com/ClickHouse/ClickHouse/pull/40078
*/
extern thread_local constinit ThreadStatus * current_thread;
2019-01-13 18:51:57 +00:00
2018-09-06 00:28:15 +00:00
/** Encapsulates all per-thread info (ProfileEvents, MemoryTracker, query_id, query context, etc.).
2019-01-13 18:51:57 +00:00
* The object must be created in thread function and destroyed in the same thread before the exit.
* It is accessed through thread-local pointer.
2018-09-06 00:28:15 +00:00
*
* This object should be used only via "CurrentThread", see CurrentThread.h
*/
2019-01-13 18:51:57 +00:00
class ThreadStatus : public boost::noncopyable
{
public:
/// Linux's PID (or TGID) (the same id is shown by ps util)
const UInt64 thread_id = 0;
2019-07-06 18:02:28 +00:00
/// Also called "nice" value. If it was changed to non-zero (when attaching query) - will be reset to zero when query is detached.
Int32 os_thread_priority = 0;
/// TODO: merge them into common entity
ProfileEvents::Counters performance_counters{VariableContext::Thread};
/// Points to performance_counters by default.
/// Could be changed to point to another object to calculate performance counters for some narrow scope.
ProfileEvents::Counters * current_performance_counters{&performance_counters};
2020-04-30 13:25:17 +00:00
MemoryTracker memory_tracker{VariableContext::Thread};
2019-07-10 18:12:50 +00:00
/// Small amount of untracked memory (per thread atomic-less counter)
Int64 untracked_memory = 0;
2020-04-30 13:25:17 +00:00
/// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters.
Int64 untracked_memory_limit = 4 * 1024 * 1024;
/// Statistics of read and write rows/bytes
Progress progress_in;
Progress progress_out;
2023-03-15 21:12:29 +00:00
private:
2023-01-23 12:45:28 +00:00
/// Group of threads, to which this thread attached
ThreadGroupPtr thread_group;
/// Is set once
ContextWeakPtr global_context;
/// Use it only from current thread
ContextWeakPtr query_context;
2023-03-15 21:12:29 +00:00
/// Is used to send logs from logs_queue to client in case of fatal errors.
using FatalErrorCallback = std::function<void()>;
FatalErrorCallback fatal_error_callback;
ThreadGroup::SharedData local_data;
2023-03-15 21:12:29 +00:00
bool performance_counters_finalized = false;
2023-03-15 21:12:29 +00:00
String query_id_from_query_context;
2021-09-02 14:27:19 +00:00
struct TimePoint
{
void setUp();
2023-03-15 21:12:29 +00:00
UInt64 nanoseconds() const;
UInt64 microseconds() const;
UInt64 seconds() const;
2023-03-15 21:12:29 +00:00
std::chrono::time_point<std::chrono::system_clock> point;
};
TimePoint query_start_time{};
// CPU and Real time query profilers
std::unique_ptr<QueryProfilerReal> query_profiler_real;
2021-08-25 00:58:49 +00:00
std::unique_ptr<QueryProfilerCPU> query_profiler_cpu;
/// Use ptr not to add extra dependencies in the header
std::unique_ptr<RUsageCounters> last_rusage;
std::unique_ptr<TasksStatsCounters> taskstats;
2023-03-22 13:51:07 +00:00
Stopwatch stopwatch{CLOCK_MONOTONIC_COARSE};
UInt64 last_performance_counters_update_time = 0;
/// See setInternalThread()
bool internal_thread = false;
/// This is helpful for cut linking dependencies for clickhouse_common_io
using Deleter = std::function<void()>;
Deleter deleter;
2023-03-15 21:12:29 +00:00
Poco::Logger * log = nullptr;
public:
ThreadStatus();
~ThreadStatus();
ThreadGroupPtr getThreadGroup() const;
2023-03-15 21:12:29 +00:00
const String & getQueryId() const;
2023-03-15 21:12:29 +00:00
ContextPtr getQueryContext() const;
ContextPtr getGlobalContext() const;
2022-03-29 17:49:42 +00:00
/// "Internal" ThreadStatus is used for materialized views for separate
/// tracking into system.query_views_log
///
/// You can have multiple internal threads, but only one non-internal with
/// the same thread_id.
///
/// "Internal" thread:
/// - cannot have query profiler
/// since the running (main query) thread should already have one
/// - should not try to obtain latest counter on detach
/// because detaching of such threads will be done from a different
/// thread_id, and some counters are not available (i.e. getrusage()),
/// but anyway they are accounted correctly in the main ThreadStatus of a
/// query.
void setInternalThread();
2021-08-13 16:32:29 +00:00
/// Attaches slave thread to existing thread group
void attachToGroup(const ThreadGroupPtr & thread_group_, bool check_detached = true);
/// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped
2023-03-15 21:12:29 +00:00
void detachFromGroup();
/// Returns pointer to the current profile counters to restore them back.
/// Note: consequent call with new scope will detach previous scope.
ProfileEvents::Counters * attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope);
2023-01-23 12:45:28 +00:00
2019-07-10 12:19:17 +00:00
void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue,
LogsLevel client_logs_level);
2023-03-15 21:12:29 +00:00
InternalTextLogsQueuePtr getInternalTextLogsQueue() const;
LogsLevel getClientLogsLevel() const;
2021-09-02 14:27:19 +00:00
void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue);
2023-03-15 21:12:29 +00:00
InternalProfileEventsQueuePtr getInternalProfileEventsQueue() const;
void attachQueryForLog(const String & query_);
const String & getQueryForLog() const;
2021-09-02 14:27:19 +00:00
/// Proper cal for fatal_error_callback
2020-06-20 11:17:15 +00:00
void onFatalError();
/// Update several ProfileEvents counters
void updatePerformanceCounters();
2023-03-22 13:51:07 +00:00
void updatePerformanceCountersIfNeeded();
/// Update ProfileEvents and dumps info to system.query_thread_log
void finalizePerformanceCounters();
/// Set the counters last usage to now
void resetPerformanceCountersLastUsage();
void logToQueryViewsLog(const ViewRuntimeData & vinfo);
2021-06-18 13:44:08 +00:00
void flushUntrackedMemory();
2023-03-15 21:12:29 +00:00
private:
void applyQuerySettings();
void initPerformanceCounters();
void initQueryProfiler();
void finalizeQueryProfiler();
2023-03-15 21:12:29 +00:00
void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database);
void attachToGroupImpl(const ThreadGroupPtr & thread_group_);
};
/**
* Creates ThreadStatus for the main thread.
*/
class MainThreadStatus : public ThreadStatus
{
public:
static MainThreadStatus & getInstance();
static ThreadStatus * get() { return main_thread; }
static bool isMainThread() { return main_thread == current_thread; }
~MainThreadStatus();
private:
MainThreadStatus();
static ThreadStatus * main_thread;
};
}