Merge pull request #47564 from CheSema/remove-redundant

remove counters for threads, fix negative counters
This commit is contained in:
Sema Checherinda 2023-03-18 02:23:52 +01:00 committed by GitHub
commit 0b641fcead
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 580 additions and 632 deletions

View File

@ -54,10 +54,10 @@ public:
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
setThreadName("UniqExactMerger");
while (true)

View File

@ -89,13 +89,13 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries
if (!--num_active_jobs)
event.notify_all();
if (async)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
try
{
if (async && thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
if (async)
setThreadName("BackupWorker");
@ -154,13 +154,13 @@ void restoreTablesData(DataRestoreTasks && tasks, ThreadPool & thread_pool)
if (!--num_active_jobs)
event.notify_all();
if (async)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
try
{
if (async && thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
if (async)
setThreadName("RestoreWorker");

View File

@ -117,6 +117,11 @@ namespace ProfileEvents
extern const Event SystemTimeMicroseconds;
}
namespace
{
constexpr UInt64 THREAD_GROUP_ID = 0;
}
namespace DB
{
@ -195,8 +200,19 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src)
}
};
std::map<Id, UInt64> rows_by_name;
for (size_t src_row = 0; src_row < src.rows(); ++src_row)
{
/// Filter out threads stats, use stats from thread group
/// Exactly stats from thread group is stored to the table system.query_log
/// The stats from threads are less useful.
/// They take more records, they need to be combined,
/// there even could be several records from one thread.
/// Server doesn't send it any more to the clients, so this code left for compatible
auto thread_id = src_array_thread_id[src_row];
if (thread_id != THREAD_GROUP_ID)
continue;
Id id{
src_column_name.getDataAt(src_row),
src_column_host_name.getDataAt(src_row),
@ -204,16 +220,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src)
rows_by_name[id] = src_row;
}
/// Filter out snapshots
std::set<size_t> thread_id_filter_mask;
for (size_t i = 0; i < src_array_thread_id.size(); ++i)
{
if (src_array_thread_id[i] != 0)
{
thread_id_filter_mask.emplace(i);
}
}
/// Merge src into dst.
for (size_t dst_row = 0; dst_row < dst_rows; ++dst_row)
{
@ -225,10 +231,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src)
if (auto it = rows_by_name.find(id); it != rows_by_name.end())
{
size_t src_row = it->second;
if (thread_id_filter_mask.contains(src_row))
{
continue;
}
dst_array_current_time[dst_row] = src_array_current_time[src_row];
@ -249,11 +251,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src)
/// Copy rows from src that dst does not contains.
for (const auto & [id, pos] : rows_by_name)
{
if (thread_id_filter_mask.contains(pos))
{
continue;
}
for (size_t col = 0; col < src.columns(); ++col)
{
mutable_columns[col]->insert((*src.getByPosition(col).column)[pos]);
@ -1080,13 +1077,18 @@ void ClientBase::onProfileEvents(Block & block)
const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds);
const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds);
HostToThreadTimesMap thread_times;
HostToTimesMap thread_times;
for (size_t i = 0; i < rows; ++i)
{
auto thread_id = array_thread_id[i];
auto host_name = host_names.getDataAt(i).toString();
if (thread_id != 0)
progress_indication.addThreadIdToList(host_name, thread_id);
/// In ProfileEvents packets thread id 0 specifies common profiling information
/// for all threads executing current query on specific host. So instead of summing per thread
/// consumption it's enough to look for data with thread id 0.
if (thread_id != THREAD_GROUP_ID)
continue;
auto event_name = names.getDataAt(i);
auto value = array_values[i];
@ -1095,11 +1097,11 @@ void ClientBase::onProfileEvents(Block & block)
continue;
if (event_name == user_time_name)
thread_times[host_name][thread_id].user_ms = value;
thread_times[host_name].user_ms = value;
else if (event_name == system_time_name)
thread_times[host_name][thread_id].system_ms = value;
thread_times[host_name].system_ms = value;
else if (event_name == MemoryTracker::USAGE_EVENT_NAME)
thread_times[host_name][thread_id].memory_usage = value;
thread_times[host_name].memory_usage = value;
}
progress_indication.updateThreadEventData(thread_times);

View File

@ -57,6 +57,23 @@ void CurrentThread::updateProgressOut(const Progress & value)
current_thread->progress_out.incrementPiecewiseAtomically(value);
}
std::shared_ptr<InternalTextLogsQueue> CurrentThread::getInternalTextLogsQueue()
{
/// NOTE: this method could be called at early server startup stage
if (unlikely(!current_thread))
return nullptr;
return current_thread->getInternalTextLogsQueue();
}
InternalProfileEventsQueuePtr CurrentThread::getInternalProfileEventsQueue()
{
if (unlikely(!current_thread))
return nullptr;
return current_thread->getInternalProfileEventsQueue();
}
void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr<InternalTextLogsQueue> & logs_queue,
LogsLevel client_logs_level)
{
@ -65,42 +82,6 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr<InternalTe
current_thread->attachInternalTextLogsQueue(logs_queue, client_logs_level);
}
void CurrentThread::setFatalErrorCallback(std::function<void()> callback)
{
/// It does not make sense to set a callback for sending logs to a client if there's no thread status
chassert(current_thread);
current_thread->setFatalErrorCallback(callback);
}
std::shared_ptr<InternalTextLogsQueue> CurrentThread::getInternalTextLogsQueue()
{
/// NOTE: this method could be called at early server startup stage
if (unlikely(!current_thread))
return nullptr;
if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died)
return nullptr;
return current_thread->getInternalTextLogsQueue();
}
void CurrentThread::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue)
{
if (unlikely(!current_thread))
return;
current_thread->attachInternalProfileEventsQueue(queue);
}
InternalProfileEventsQueuePtr CurrentThread::getInternalProfileEventsQueue()
{
if (unlikely(!current_thread))
return nullptr;
if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died)
return nullptr;
return current_thread->getInternalProfileEventsQueue();
}
ThreadGroupStatusPtr CurrentThread::getGroup()
{
@ -110,4 +91,12 @@ ThreadGroupStatusPtr CurrentThread::getGroup()
return current_thread->getThreadGroup();
}
std::string_view CurrentThread::getQueryId()
{
if (unlikely(!current_thread))
return {};
return current_thread->getQueryId();
}
}

View File

@ -5,6 +5,7 @@
#include <memory>
#include <string>
#include <string_view>
namespace ProfileEvents
@ -48,7 +49,7 @@ public:
static void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue);
static InternalProfileEventsQueuePtr getInternalProfileEventsQueue();
static void setFatalErrorCallback(std::function<void()> callback);
static void attachQueryForLog(const String & query_);
/// Makes system calls to update ProfileEvents that contain info from rusage and taskstats
static void updatePerformanceCounters();
@ -65,31 +66,20 @@ public:
static void updateProgressIn(const Progress & value);
static void updateProgressOut(const Progress & value);
/// Query management:
/// Call from master thread as soon as possible (e.g. when thread accepted connection)
static void initializeQuery();
/// You must call one of these methods when create a query child thread:
/// Add current thread to a group associated with the thread group
static void attachTo(const ThreadGroupStatusPtr & thread_group);
static void attachToGroup(const ThreadGroupStatusPtr & thread_group);
/// Is useful for a ThreadPool tasks
static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group);
static void attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group);
/// Non-master threads call this method in destructor automatically
static void detachFromGroupIfNotDetached();
/// Update ProfileEvents and dumps info to system.query_thread_log
static void finalizePerformanceCounters();
/// Returns a non-empty string if the thread is attached to a query
static std::string_view getQueryId()
{
if (unlikely(!current_thread))
return {};
return current_thread->getQueryId();
}
/// Non-master threads call this method in destructor automatically
static void detachQuery();
static void detachQueryIfNotDetached();
static std::string_view getQueryId();
/// Initializes query with current thread as master thread in constructor, and detaches it in destructor
struct QueryScope : private boost::noncopyable
@ -101,13 +91,6 @@ public:
void logPeakMemoryUsage();
bool log_peak_memory_usage_in_destructor = true;
};
private:
static void defaultThreadDeleter();
/// Sets query_context for current thread group
/// Can by used only through QueryScope
static void attachQueryContext(ContextPtr query_context);
};
}

View File

@ -15,24 +15,6 @@
/// http://en.wikipedia.org/wiki/ANSI_escape_code
#define CLEAR_TO_END_OF_LINE "\033[K"
namespace
{
constexpr UInt64 ALL_THREADS = 0;
UInt64 aggregateCPUUsageNs(DB::ThreadIdToTimeMap times)
{
constexpr UInt64 us_to_ns = 1000;
return us_to_ns * std::accumulate(times.begin(), times.end(), 0ull,
[](UInt64 acc, const auto & elem)
{
if (elem.first == ALL_THREADS)
return acc;
return acc + elem.second.time();
});
}
}
namespace DB
{
@ -58,7 +40,7 @@ void ProgressIndication::resetProgress()
{
std::lock_guard lock(profile_events_mutex);
cpu_usage_meter.reset(getElapsedNanoseconds());
thread_data.clear();
hosts_data.clear();
}
}
@ -71,25 +53,17 @@ void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, Writ
});
}
void ProgressIndication::addThreadIdToList(String const & host, UInt64 thread_id)
void ProgressIndication::updateThreadEventData(HostToTimesMap & new_hosts_data)
{
std::lock_guard lock(profile_events_mutex);
auto & thread_to_times = thread_data[host];
if (thread_to_times.contains(thread_id))
return;
thread_to_times[thread_id] = {};
}
void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread_data)
{
std::lock_guard lock(profile_events_mutex);
constexpr UInt64 us_to_ns = 1000;
UInt64 total_cpu_ns = 0;
for (auto & new_host_map : new_thread_data)
for (auto & new_host : new_hosts_data)
{
total_cpu_ns += aggregateCPUUsageNs(new_host_map.second);
thread_data[new_host_map.first] = std::move(new_host_map.second);
total_cpu_ns += us_to_ns * new_host.second.time();
hosts_data[new_host.first] = new_host.second;
}
cpu_usage_meter.add(getElapsedNanoseconds(), total_cpu_ns);
}
@ -104,16 +78,10 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const
{
std::lock_guard lock(profile_events_mutex);
return std::accumulate(thread_data.cbegin(), thread_data.cend(), MemoryUsage{},
return std::accumulate(hosts_data.cbegin(), hosts_data.cend(), MemoryUsage{},
[](MemoryUsage const & acc, auto const & host_data)
{
UInt64 host_usage = 0;
// In ProfileEvents packets thread id 0 specifies common profiling information
// for all threads executing current query on specific host. So instead of summing per thread
// memory consumption it's enough to look for data with thread id 0.
if (auto it = host_data.second.find(ALL_THREADS); it != host_data.second.end())
host_usage = it->second.memory_usage;
UInt64 host_usage = host_data.second.memory_usage;
return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)};
});
}

View File

@ -24,8 +24,7 @@ struct ThreadEventData
UInt64 memory_usage = 0;
};
using ThreadIdToTimeMap = std::unordered_map<UInt64, ThreadEventData>;
using HostToThreadTimesMap = std::unordered_map<String, ThreadIdToTimeMap>;
using HostToTimesMap = std::unordered_map<String, ThreadEventData>;
class ProgressIndication
{
@ -56,9 +55,7 @@ public:
/// How much seconds passed since query execution start.
double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; }
void addThreadIdToList(String const & host, UInt64 thread_id);
void updateThreadEventData(HostToThreadTimesMap & new_thread_data);
void updateThreadEventData(HostToTimesMap & new_hosts_data);
private:
double getCPUUsage();
@ -91,7 +88,7 @@ private:
bool write_progress_on_update = false;
EventRateMeter cpu_usage_meter{static_cast<double>(clock_gettime_ns()), 2'000'000'000 /*ns*/}; // average cpu utilization last 2 second
HostToThreadTimesMap thread_data;
HostToTimesMap hosts_data;
/// In case of all of the above:
/// - clickhouse-local
/// - input_format_parallel_parsing=true
@ -99,7 +96,7 @@ private:
///
/// It is possible concurrent access to the following:
/// - writeProgress() (class properties) (guarded with progress_mutex)
/// - thread_data/cpu_usage_meter (guarded with profile_events_mutex)
/// - hosts_data/cpu_usage_meter (guarded with profile_events_mutex)
mutable std::mutex profile_events_mutex;
mutable std::mutex progress_mutex;
};

View File

@ -1,6 +1,5 @@
#include <Common/Exception.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/QueryProfiler.h>
#include <Common/ThreadStatus.h>
#include <base/errnoToString.h>
@ -11,19 +10,12 @@
#include <base/getPageSize.h>
#include <csignal>
#include <mutex>
#include <sys/mman.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
thread_local ThreadStatus constinit * current_thread = nullptr;
#if !defined(SANITIZER)
@ -71,24 +63,9 @@ static thread_local ThreadStack alt_stack;
static thread_local bool has_alt_stack = false;
#endif
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;
}
ThreadGroupStatus::ThreadGroupStatus()
: master_thread_id(CurrentThread::get().thread_id)
{}
ThreadStatus::ThreadStatus()
: thread_id{getThreadId()}
@ -144,6 +121,63 @@ ThreadStatus::ThreadStatus()
#endif
}
ThreadGroupStatusPtr ThreadStatus::getThreadGroup() const
{
return thread_group;
}
const String & ThreadStatus::getQueryId() const
{
return query_id_from_query_context;
}
ContextPtr ThreadStatus::getQueryContext() const
{
return query_context.lock();
}
ContextPtr ThreadStatus::getGlobalContext() const
{
return global_context.lock();
}
void ThreadGroupStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level)
{
std::lock_guard lock(mutex);
shared_data.logs_queue_ptr = logs_queue;
shared_data.client_logs_level = logs_level;
}
void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue,
LogsLevel logs_level)
{
local_data.logs_queue_ptr = logs_queue;
local_data.client_logs_level = logs_level;
if (thread_group)
thread_group->attachInternalTextLogsQueue(logs_queue, logs_level);
}
InternalTextLogsQueuePtr ThreadStatus::getInternalTextLogsQueue() const
{
return local_data.logs_queue_ptr.lock();
}
InternalProfileEventsQueuePtr ThreadStatus::getInternalProfileEventsQueue() const
{
return local_data.profile_queue_ptr.lock();
}
const String & ThreadStatus::getQueryForLog() const
{
return local_data.query_for_logs;
}
LogsLevel ThreadStatus::getClientLogsLevel() const
{
return local_data.client_logs_level;
}
void ThreadStatus::flushUntrackedMemory()
{
if (untracked_memory == 0)
@ -157,24 +191,11 @@ ThreadStatus::~ThreadStatus()
{
flushUntrackedMemory();
if (thread_group)
{
ThreadGroupStatus::ProfileEventsCountersAndMemory counters
{
performance_counters.getPartiallyAtomicSnapshot(),
memory_tracker.get(),
thread_id
};
std::lock_guard guard(thread_group->mutex);
thread_group->finished_threads_counters_memory.emplace_back(std::move(counters));
thread_group->threads.erase(this);
}
/// It may cause segfault if query_context was destroyed, but was not detached
auto query_context_ptr = query_context.lock();
assert((!query_context_ptr && query_id.empty()) || (query_context_ptr && query_id == query_context_ptr->getCurrentQueryId()));
assert((!query_context_ptr && getQueryId().empty()) || (query_context_ptr && getQueryId() == query_context_ptr->getCurrentQueryId()));
/// detachGroup if it was attached
if (deleter)
deleter();
@ -198,71 +219,25 @@ void ThreadStatus::updatePerformanceCounters()
}
}
void ThreadStatus::assertState(ThreadState permitted_state, const char * description) const
{
if (getCurrentState() == permitted_state)
return;
if (description)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", getCurrentState(), description);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}", getCurrentState());
}
void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue,
LogsLevel client_logs_level)
{
logs_queue_ptr = logs_queue;
if (!thread_group)
return;
std::lock_guard lock(thread_group->mutex);
thread_group->logs_queue_ptr = logs_queue;
thread_group->client_logs_level = client_logs_level;
}
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;
}
void ThreadStatus::setFatalErrorCallback(std::function<void()> callback)
{
/// It does not make sense to set a callback for sending logs to a client if there's no thread group
chassert(thread_group);
std::lock_guard lock(thread_group->mutex);
fatal_error_callback = std::move(callback);
thread_group->fatal_error_callback = fatal_error_callback;
}
void ThreadStatus::onFatalError()
{
/// No thread group - no callback
if (!thread_group)
return;
std::lock_guard lock(thread_group->mutex);
if (fatal_error_callback)
fatal_error_callback();
}
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;

View File

@ -57,46 +57,66 @@ using ThreadStatusPtr = ThreadStatus *;
* Create via CurrentThread::initializeQuery (for queries) or directly (for various background tasks).
* Use via CurrentThread::getGroup.
*/
class ThreadGroupStatus;
using ThreadGroupStatusPtr = std::shared_ptr<ThreadGroupStatus>;
class ThreadGroupStatus
{
public:
struct ProfileEventsCountersAndMemory
{
ProfileEvents::Counters::Snapshot counters;
Int64 memory_usage;
UInt64 thread_id;
};
ThreadGroupStatus();
using FatalErrorCallback = std::function<void()>;
ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {});
mutable std::mutex mutex;
/// The first thread created this thread group
const UInt64 master_thread_id;
/// Set up at creation, no race when reading
const ContextWeakPtr query_context;
const ContextWeakPtr global_context;
const FatalErrorCallback fatal_error_callback;
ProfileEvents::Counters performance_counters{VariableContext::Process};
MemoryTracker memory_tracker{VariableContext::Process};
ContextWeakPtr query_context;
ContextWeakPtr global_context;
struct SharedData
{
InternalProfileEventsQueueWeakPtr profile_queue_ptr;
InternalTextLogsQueueWeakPtr logs_queue_ptr;
InternalProfileEventsQueueWeakPtr profile_queue_ptr;
std::function<void()> fatal_error_callback;
InternalTextLogsQueueWeakPtr logs_queue_ptr;
LogsLevel client_logs_level = LogsLevel::none;
String query_for_logs;
UInt64 normalized_query_hash = 0;
};
SharedData getSharedData()
{
/// Critical section for making the copy of shared_data
std::lock_guard lock(mutex);
return shared_data;
}
/// 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);
/// When new query starts, new thread group is created for it, current thread becomes master thread of the query
static ThreadGroupStatusPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {});
std::vector<UInt64> getInvolvedThreadIds() const;
void linkThread(UInt64 thread_it);
private:
mutable std::mutex mutex;
/// Set up at creation, no race when reading
SharedData shared_data;
/// Set of all thread ids which has been attached to the group
std::unordered_set<UInt64> thread_ids;
std::unordered_set<ThreadStatusPtr> threads;
/// The first thread created this thread group
UInt64 master_thread_id = 0;
LogsLevel client_logs_level = LogsLevel::none;
String query;
UInt64 normalized_query_hash = 0;
std::vector<ProfileEventsCountersAndMemory> finished_threads_counters_memory;
std::vector<ProfileEventsCountersAndMemory> getProfileEventsCountersAndMemoryForThreads();
};
using ThreadGroupStatusPtr = std::shared_ptr<ThreadGroupStatus>;
/**
* 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.
@ -124,12 +144,11 @@ public:
/// 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};
MemoryTracker memory_tracker{VariableContext::Thread};
MemoryTracker memory_tracker{VariableContext::Thread};
/// Small amount of untracked memory (per thread atomic-less counter)
Int64 untracked_memory = 0;
/// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters.
@ -139,93 +158,70 @@ public:
Progress progress_in;
Progress progress_out;
using Deleter = std::function<void()>;
Deleter deleter;
protected:
private:
/// Group of threads, to which this thread attached
ThreadGroupStatusPtr thread_group;
std::atomic<int> thread_state{ThreadState::DetachedFromQuery};
/// Is set once
ContextWeakPtr global_context;
/// Use it only from current thread
ContextWeakPtr query_context;
String query_id;
/// Is used to send logs from logs_queue to client in case of fatal errors.
using FatalErrorCallback = std::function<void()>;
FatalErrorCallback fatal_error_callback;
/// A logs queue used by TCPHandler to pass logs to a client
InternalTextLogsQueueWeakPtr logs_queue_ptr;
InternalProfileEventsQueueWeakPtr profile_queue_ptr;
ThreadGroupStatus::SharedData local_data;
bool performance_counters_finalized = false;
UInt64 query_start_time_nanoseconds = 0;
UInt64 query_start_time_microseconds = 0;
time_t query_start_time = 0;
size_t queries_started = 0;
String query_id_from_query_context;
/// Requires access to query_id.
friend class MemoryTrackerThreadSwitcher;
void setQueryId(const String & query_id_)
{
query_id_from_query_context = query_id_;
}
struct TimePoint
{
void setUp();
UInt64 nanoseconds() const;
UInt64 microseconds() const;
UInt64 seconds() const;
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;
std::unique_ptr<QueryProfilerCPU> query_profiler_cpu;
Poco::Logger * log = nullptr;
friend class CurrentThread;
/// Use ptr not to add extra dependencies in the header
std::unique_ptr<RUsageCounters> last_rusage;
std::unique_ptr<TasksStatsCounters> taskstats;
/// Is used to send logs from logs_queue to client in case of fatal errors.
std::function<void()> fatal_error_callback;
/// See setInternalThread()
bool internal_thread = false;
/// Requires access to query_id.
friend class MemoryTrackerThreadSwitcher;
void setQueryId(const String & query_id_)
{
query_id = query_id_;
}
/// This is helpful for cut linking dependencies for clickhouse_common_io
using Deleter = std::function<void()>;
Deleter deleter;
Poco::Logger * log = nullptr;
public:
ThreadStatus();
~ThreadStatus();
ThreadGroupStatusPtr getThreadGroup() const
{
return thread_group;
}
ThreadGroupStatusPtr getThreadGroup() const;
enum ThreadState
{
DetachedFromQuery = 0, /// We just created thread or it is a background thread
AttachedToQuery, /// Thread executes enqueued query
Died, /// Thread does not exist
};
const String & getQueryId() const;
int getCurrentState() const
{
return thread_state.load(std::memory_order_relaxed);
}
std::string_view getQueryId() const
{
return query_id;
}
auto getQueryContext() const
{
return query_context.lock();
}
auto getGlobalContext() const
{
return global_context.lock();
}
ContextPtr getQueryContext() const;
ContextPtr getGlobalContext() const;
/// "Internal" ThreadStatus is used for materialized views for separate
/// tracking into system.query_views_log
@ -243,39 +239,30 @@ public:
/// query.
void setInternalThread();
/// Starts new query and create new thread group for it, current thread becomes master thread of the query
void initializeQuery();
/// Attaches slave thread to existing thread group
void attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true);
void attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true);
/// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped
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);
InternalTextLogsQueuePtr getInternalTextLogsQueue() const
{
return thread_state == Died ? nullptr : logs_queue_ptr.lock();
}
void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue,
LogsLevel client_logs_level);
InternalProfileEventsQueuePtr getInternalProfileEventsQueue() const
{
return thread_state == Died ? nullptr : profile_queue_ptr.lock();
}
InternalTextLogsQueuePtr getInternalTextLogsQueue() const;
LogsLevel getClientLogsLevel() const;
void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue);
InternalProfileEventsQueuePtr getInternalProfileEventsQueue() const;
/// Callback that is used to trigger sending fatal error messages to client.
void setFatalErrorCallback(std::function<void()> callback);
void attachQueryForLog(const String & query_);
const String & getQueryForLog() const;
/// Proper cal for fatal_error_callback
void onFatalError();
/// Sets query context for current master thread and its thread group
/// NOTE: query_context have to be alive until detachQuery() is called
void attachQueryContext(ContextPtr query_context);
/// Update several ProfileEvents counters
void updatePerformanceCounters();
@ -285,14 +272,11 @@ public:
/// Set the counters last usage to now
void resetPerformanceCountersLastUsage();
/// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped
void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false);
void logToQueryViewsLog(const ViewRuntimeData & vinfo);
void flushUntrackedMemory();
protected:
private:
void applyQuerySettings();
void initPerformanceCounters();
@ -301,14 +285,9 @@ protected:
void finalizeQueryProfiler();
void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point<std::chrono::system_clock> now);
void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database);
void assertState(ThreadState permitted_state, const char * description = nullptr) const;
private:
void setupState(const ThreadGroupStatusPtr & thread_group_);
void attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_);
};
/**

View File

@ -34,7 +34,7 @@ namespace
{ \
auto _logger = ::getLogger(logger); \
const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \
(DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \
(DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \
if (_is_clients_log || _logger->is((PRIORITY))) \
{ \
std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \

View File

@ -310,12 +310,8 @@ private:
/// It will allow client to see failure messages directly.
if (thread_ptr)
{
query_id = std::string(thread_ptr->getQueryId());
if (auto thread_group = thread_ptr->getThreadGroup())
{
query = DB::toOneLineQuery(thread_group->query);
}
query_id = thread_ptr->getQueryId();
query = thread_ptr->getQueryForLog();
if (auto logs_queue = thread_ptr->getInternalTextLogsQueue())
{

View File

@ -75,7 +75,7 @@ public:
pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()]
{
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
setThreadName("HashedDictLoad");
threadWorker(shard);
@ -224,7 +224,7 @@ HashedDictionary<dictionary_key_type, sparse, sharded>::~HashedDictionary()
pool.trySchedule([&container, thread_group = CurrentThread::getGroup()]
{
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
setThreadName("HashedDictDtor");
if constexpr (sparse)

View File

@ -2309,10 +2309,10 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
BlocksList blocks;
while (true)
@ -3030,10 +3030,10 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
for (Block & block : bucket_to_blocks[bucket])
{

View File

@ -971,11 +971,11 @@ private:
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
LOG_TRACE(log, "Start loading object '{}'", name);
try

View File

@ -203,10 +203,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
ProcessListForUser & user_process_list = user_process_list_it->second;
/// Actualize thread group info
CurrentThread::attachQueryForLog(query_);
auto thread_group = CurrentThread::getGroup();
if (thread_group)
{
std::lock_guard lock_thread_group(thread_group->mutex);
thread_group->performance_counters.setParent(&user_process_list.user_performance_counters);
thread_group->memory_tracker.setParent(&user_process_list.user_memory_tracker);
if (user_process_list.user_temp_data_on_disk)
@ -214,8 +214,6 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
query_context->setTempDataOnDisk(std::make_shared<TemporaryDataOnDiskScope>(
user_process_list.user_temp_data_on_disk, settings.max_temporary_data_on_disk_size_for_query));
}
thread_group->query = query_;
thread_group->normalized_query_hash = normalizedQueryHash<false>(query_);
/// Set query-level memory trackers
thread_group->memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage);
@ -576,10 +574,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
res.peak_memory_usage = thread_group->memory_tracker.getPeak();
if (get_thread_list)
{
std::lock_guard lock(thread_group->mutex);
res.thread_ids.assign(thread_group->thread_ids.begin(), thread_group->thread_ids.end());
}
res.thread_ids = thread_group->getInvolvedThreadIds();
if (get_profile_events)
res.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(thread_group->performance_counters.getPartiallyAtomicSnapshot());

View File

@ -113,34 +113,10 @@ void getProfileEvents(
block = std::move(temp_columns);
MutableColumns columns = block.mutateColumns();
auto thread_group = CurrentThread::getGroup();
auto const current_thread_id = CurrentThread::get().thread_id;
std::vector<ProfileEventsSnapshot> snapshots;
ThreadIdToCountersSnapshot new_snapshots;
ProfileEventsSnapshot group_snapshot;
{
auto stats = thread_group->getProfileEventsCountersAndMemoryForThreads();
snapshots.reserve(stats.size());
for (auto & stat : stats)
{
auto const thread_id = stat.thread_id;
if (thread_id == current_thread_id)
continue;
auto current_time = time(nullptr);
auto previous_snapshot = last_sent_snapshots.find(thread_id);
auto increment =
previous_snapshot != last_sent_snapshots.end()
? CountersIncrement(stat.counters, previous_snapshot->second)
: CountersIncrement(stat.counters);
snapshots.push_back(ProfileEventsSnapshot{
thread_id,
std::move(increment),
stat.memory_usage,
current_time
});
new_snapshots[thread_id] = std::move(stat.counters);
}
group_snapshot.thread_id = 0;
group_snapshot.current_time = time(nullptr);
group_snapshot.memory_usage = thread_group->memory_tracker.get();
@ -154,11 +130,6 @@ void getProfileEvents(
}
last_sent_snapshots = std::move(new_snapshots);
for (auto & snapshot : snapshots)
{
dumpProfileEvents(snapshot, columns, server_display_name);
dumpMemoryTracker(snapshot, columns, server_display_name);
}
dumpProfileEvents(group_snapshot, columns, server_display_name);
dumpMemoryTracker(group_snapshot, columns, server_display_name);

View File

@ -9,6 +9,7 @@
#include <Interpreters/QueryViewsLog.h>
#include <Interpreters/TraceCollector.h>
#include <Parsers/formatAST.h>
#include <Parsers/queryNormalization.h>
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/ProfileEvents.h>
@ -40,13 +41,96 @@ namespace ErrorCodes
extern const int CANNOT_SET_THREAD_PRIORITY;
}
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
{
std::vector<UInt64> res;
{
std::lock_guard lock(mutex);
res.assign(thread_ids.begin(), thread_ids.end());
}
return res;
}
void ThreadGroupStatus::linkThread(UInt64 thread_it)
{
std::lock_guard lock(mutex);
thread_ids.insert(thread_it);
}
ThreadGroupStatusPtr ThreadGroupStatus::createForQuery(ContextPtr query_context_, std::function<void()> fatal_error_callback_)
{
auto group = std::make_shared<ThreadGroupStatus>(query_context_, std::move(fatal_error_callback_));
group->memory_tracker.setDescription("(for query)");
return group;
}
void ThreadGroupStatus::attachQueryForLog(const String & query_, UInt64 normalized_hash)
{
auto hash = normalized_hash ? normalized_hash : normalizedQueryHash<false>(query_);
std::lock_guard lock(mutex);
shared_data.query_for_logs = query_;
shared_data.normalized_query_hash = hash;
}
void ThreadStatus::attachQueryForLog(const String & query_)
{
local_data.query_for_logs = query_;
local_data.normalized_query_hash = normalizedQueryHash<false>(query_);
if (!thread_group)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id);
thread_group->attachQueryForLog(local_data.query_for_logs, local_data.normalized_query_hash);
}
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);
local_data.profile_queue_ptr = profile_queue;
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_);
}
void ThreadStatus::applyQuerySettings()
{
auto query_context_ptr = query_context.lock();
assert(query_context_ptr);
if (!query_context_ptr)
return;
const Settings & settings = query_context_ptr->getSettingsRef();
query_id = query_context_ptr->getCurrentQueryId();
query_id_from_query_context = query_context_ptr->getCurrentQueryId();
initQueryProfiler();
untracked_memory_limit = settings.max_untracked_memory;
@ -68,67 +152,64 @@ void ThreadStatus::applyQuerySettings()
#endif
}
void ThreadStatus::attachQueryContext(ContextPtr query_context_)
void ThreadStatus::attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_)
{
query_context = query_context_;
if (global_context.expired())
global_context = query_context_->getGlobalContext();
if (thread_group)
{
std::lock_guard lock(thread_group->mutex);
thread_group->query_context = query_context;
if (thread_group->global_context.expired())
thread_group->global_context = global_context;
}
applyQuerySettings();
}
void CurrentThread::defaultThreadDeleter()
{
if (unlikely(!current_thread))
return;
current_thread->detachQuery(true, true);
}
void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_)
{
assertState(ThreadState::DetachedFromQuery, __PRETTY_FUNCTION__);
/// Attach or init current thread to thread group and copy useful information from it
thread_group = thread_group_;
thread_group->linkThread(thread_id);
performance_counters.setParent(&thread_group->performance_counters);
memory_tracker.setParent(&thread_group->memory_tracker);
{
std::lock_guard lock(thread_group->mutex);
query_context = thread_group->query_context;
global_context = thread_group->global_context;
/// NOTE: thread may be attached multiple times if it is reused from a thread pool.
thread_group->thread_ids.insert(thread_id);
thread_group->threads.insert(this);
fatal_error_callback = thread_group->fatal_error_callback;
logs_queue_ptr = thread_group->logs_queue_ptr;
fatal_error_callback = thread_group->fatal_error_callback;
query_context = thread_group->query_context;
profile_queue_ptr = thread_group->profile_queue_ptr;
if (global_context.expired())
global_context = thread_group->global_context;
}
if (auto query_context_ptr = query_context.lock())
{
applyQuerySettings();
}
local_data = thread_group->getSharedData();
applyQuerySettings();
initPerformanceCounters();
}
thread_state = ThreadState::AttachedToQuery;
void ThreadStatus::detachFromGroup()
{
if (!thread_group)
return;
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
/// flash untracked memory before resetting memory_tracker parent
flushUntrackedMemory();
finalizeQueryProfiler();
finalizePerformanceCounters();
performance_counters.setParent(&ProfileEvents::global_counters);
memory_tracker.reset();
memory_tracker.setParent(thread_group->memory_tracker.getParent());
thread_group.reset();
query_id_from_query_context.clear();
query_context.reset();
local_data = {};
fatal_error_callback = {};
#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
}
void ThreadStatus::setInternalThread()
@ -137,28 +218,19 @@ void ThreadStatus::setInternalThread()
internal_thread = true;
}
void ThreadStatus::initializeQuery()
void ThreadStatus::attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached)
{
setupState(std::make_shared<ThreadGroupStatus>());
/// No need to lock on mutex here
thread_group->memory_tracker.setDescription("(for query)");
thread_group->master_thread_id = thread_id;
}
void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached)
{
if (thread_state == ThreadState::AttachedToQuery)
{
if (check_detached)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached");
return;
}
if (thread_group && check_detached)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached");
if (!thread_group_)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to attach to nullptr thread group");
setupState(thread_group_);
if (thread_group)
return;
deleter = [this] () { detachFromGroup(); };
attachToGroupImpl(thread_group_);
}
ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope)
@ -178,6 +250,26 @@ ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents
return prev_counters;
}
void ThreadStatus::TimePoint::setUp()
{
point = std::chrono::system_clock::now();
}
UInt64 ThreadStatus::TimePoint::nanoseconds() const
{
return timeInNanoseconds(point);
}
UInt64 ThreadStatus::TimePoint::microseconds() const
{
return timeInMicroseconds(point);
}
UInt64 ThreadStatus::TimePoint::seconds() const
{
return timeInSeconds(point);
}
void ThreadStatus::initPerformanceCounters()
{
performance_counters_finalized = false;
@ -188,16 +280,9 @@ void ThreadStatus::initPerformanceCounters()
memory_tracker.resetCounters();
memory_tracker.setDescription("(for thread)");
// query_start_time_{microseconds, nanoseconds} are all constructed from the same time point
// to ensure that they are all equal up to the precision of a second.
const auto now = std::chrono::system_clock::now();
query_start_time.setUp();
query_start_time_nanoseconds = timeInNanoseconds(now);
query_start_time = timeInSeconds(now);
query_start_time_microseconds = timeInMicroseconds(now);
++queries_started;
// query_start_time_nanoseconds cannot be used here since RUsageCounters expect CLOCK_MONOTONIC
// query_start_time.nanoseconds cannot be used here since RUsageCounters expect CLOCK_MONOTONIC
*last_rusage = RUsageCounters::current();
if (!internal_thread)
@ -272,11 +357,11 @@ void ThreadStatus::finalizePerformanceCounters()
if (settings.log_queries && settings.log_query_threads)
{
const auto now = std::chrono::system_clock::now();
Int64 query_duration_ms = (timeInMicroseconds(now) - query_start_time_microseconds) / 1000;
Int64 query_duration_ms = std::chrono::duration_cast<std::chrono::microseconds>(now - query_start_time.point).count();
if (query_duration_ms >= settings.log_queries_min_query_duration_ms.totalMilliseconds())
{
if (auto thread_log = global_context_ptr->getQueryThreadLog())
logToQueryThreadLog(*thread_log, query_context_ptr->getCurrentDatabase(), now);
logToQueryThreadLog(*thread_log, query_context_ptr->getCurrentDatabase());
}
}
}
@ -331,77 +416,20 @@ void ThreadStatus::finalizeQueryProfiler()
query_profiler_cpu.reset();
}
void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
{
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery)
{
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;
return;
}
assertState(ThreadState::AttachedToQuery, __PRETTY_FUNCTION__);
finalizeQueryProfiler();
finalizePerformanceCounters();
/// Detach from thread group
{
std::lock_guard guard(thread_group->mutex);
thread_group->threads.erase(this);
}
performance_counters.setParent(&ProfileEvents::global_counters);
flushUntrackedMemory();
memory_tracker.reset();
memory_tracker.setParent(thread_group->memory_tracker.getParent());
query_id.clear();
query_context.reset();
/// The memory of thread_group->finished_threads_counters_memory is temporarily moved to this vector, which is deallocated out of critical section.
std::vector<ThreadGroupStatus::ProfileEventsCountersAndMemory> move_to_temp;
/// Avoid leaking of ThreadGroupStatus::finished_threads_counters_memory
/// (this is in case someone uses system thread but did not call getProfileEventsCountersAndMemoryForThreads())
{
std::lock_guard guard(thread_group->mutex);
move_to_temp = std::move(thread_group->finished_threads_counters_memory);
}
thread_group.reset();
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;
#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
}
void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point<std::chrono::system_clock> now)
void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database)
{
QueryThreadLogElement elem;
// 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.
auto current_time = timeInSeconds(now);
auto current_time_microseconds = timeInMicroseconds(now);
TimePoint current_time;
current_time.setUp();
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = query_start_time;
elem.query_start_time_microseconds = query_start_time_microseconds;
elem.query_duration_ms = (timeInNanoseconds(now) - query_start_time_nanoseconds) / 1000000U;
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();
elem.read_rows = progress_in.read_rows.load(std::memory_order_relaxed);
elem.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed);
@ -417,13 +445,9 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String
elem.current_database = current_database;
if (thread_group)
{
{
std::lock_guard lock(thread_group->mutex);
elem.master_thread_id = thread_group->master_thread_id;
elem.query = thread_group->query;
elem.normalized_query_hash = thread_group->normalized_query_hash;
}
elem.master_thread_id = thread_group->master_thread_id;
elem.query = local_data.query_for_logs;
elem.normalized_query_hash = local_data.normalized_query_hash;
}
auto query_context_ptr = query_context.lock();
@ -457,6 +481,7 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo)
auto query_context_ptr = query_context.lock();
if (!query_context_ptr)
return;
auto views_log = query_context_ptr->getQueryViewsLog();
if (!views_log)
return;
@ -467,7 +492,7 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo)
element.event_time_microseconds = timeInMicroseconds(vinfo.runtime_stats->event_time);
element.view_duration_ms = vinfo.runtime_stats->elapsed_ms;
element.initial_query_id = query_id;
element.initial_query_id = query_id_from_query_context;
element.view_name = vinfo.table_id.getFullTableName();
element.view_uuid = vinfo.table_id.uuid;
element.view_type = vinfo.runtime_stats->type;
@ -475,16 +500,14 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo)
element.view_query = getCleanQueryAst(vinfo.query, query_context_ptr);
element.view_target = vinfo.runtime_stats->target_name;
auto events = std::make_shared<ProfileEvents::Counters::Snapshot>(performance_counters.getPartiallyAtomicSnapshot());
element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed);
element.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed);
element.written_rows = progress_out.written_rows.load(std::memory_order_relaxed);
element.written_bytes = progress_out.written_bytes.load(std::memory_order_relaxed);
element.peak_memory_usage = memory_tracker.getPeak() > 0 ? memory_tracker.getPeak() : 0;
if (query_context_ptr->getSettingsRef().log_profile_events != 0)
{
element.profile_counters = events;
}
element.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(
performance_counters.getPartiallyAtomicSnapshot());
element.status = vinfo.runtime_stats->event_status;
element.exception_code = 0;
@ -499,35 +522,18 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo)
views_log->add(element);
}
void CurrentThread::initializeQuery()
void CurrentThread::attachToGroup(const ThreadGroupStatusPtr & thread_group)
{
if (unlikely(!current_thread))
return;
current_thread->initializeQuery();
current_thread->deleter = CurrentThread::defaultThreadDeleter;
current_thread->attachToGroup(thread_group, true);
}
void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group)
void CurrentThread::attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group)
{
if (unlikely(!current_thread))
return;
current_thread->attachQuery(thread_group, true);
current_thread->deleter = CurrentThread::defaultThreadDeleter;
}
void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group)
{
if (unlikely(!current_thread))
return;
current_thread->attachQuery(thread_group, false);
current_thread->deleter = CurrentThread::defaultThreadDeleter;
}
void CurrentThread::attachQueryContext(ContextPtr query_context)
{
if (unlikely(!current_thread))
return;
current_thread->attachQueryContext(query_context);
current_thread->attachToGroup(thread_group, false);
}
void CurrentThread::finalizePerformanceCounters()
@ -537,28 +543,20 @@ void CurrentThread::finalizePerformanceCounters()
current_thread->finalizePerformanceCounters();
}
void CurrentThread::detachQuery()
void CurrentThread::detachFromGroupIfNotDetached()
{
if (unlikely(!current_thread))
return;
current_thread->detachQuery(false);
current_thread->detachFromGroup();
}
void CurrentThread::detachQueryIfNotDetached()
{
if (unlikely(!current_thread))
return;
current_thread->detachQuery(true);
}
CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::function<void()> fatal_error_callback)
{
CurrentThread::initializeQuery();
CurrentThread::attachQueryContext(query_context);
if (!query_context->hasQueryContext())
query_context->makeQueryContext();
setFatalErrorCallback(fatal_error_callback);
auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback));
CurrentThread::attachToGroup(group);
}
CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function<void()> fatal_error_callback)
@ -567,9 +565,8 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function<vo
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Cannot initialize query scope without query context");
CurrentThread::initializeQuery();
CurrentThread::attachQueryContext(query_context);
setFatalErrorCallback(fatal_error_callback);
auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback));
CurrentThread::attachToGroup(group);
}
void CurrentThread::QueryScope::logPeakMemoryUsage()
@ -589,8 +586,7 @@ CurrentThread::QueryScope::~QueryScope()
if (log_peak_memory_usage_in_destructor)
logPeakMemoryUsage();
setFatalErrorCallback({});
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
}
catch (...)
{

View File

@ -131,6 +131,7 @@ TEST_F(FileCacheTest, get)
auto query_context = DB::Context::createCopy(getContext().context);
query_context->makeQueryContext();
query_context->setCurrentQueryId(query_id);
chassert(&DB::CurrentThread::get() == &thread_status);
DB::CurrentThread::QueryScope query_scope_holder(query_context);
DB::FileCacheSettings settings;
@ -398,8 +399,8 @@ TEST_F(FileCacheTest, get)
auto query_context_1 = DB::Context::createCopy(getContext().context);
query_context_1->makeQueryContext();
query_context_1->setCurrentQueryId("query_id_1");
chassert(&DB::CurrentThread::get() == &thread_status_1);
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
thread_status_1.attachQueryContext(query_context_1);
auto holder_2 = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] once again.
auto segments_2 = fromHolder(holder_2);
@ -467,8 +468,8 @@ TEST_F(FileCacheTest, get)
auto query_context_1 = DB::Context::createCopy(getContext().context);
query_context_1->makeQueryContext();
query_context_1->setCurrentQueryId("query_id_1");
chassert(&DB::CurrentThread::get() == &thread_status_1);
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
thread_status_1.attachQueryContext(query_context_1);
auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again
auto segments_2 = fromHolder(*holder);

View File

@ -22,7 +22,7 @@ ThreadPoolCallbackRunner<Result, Callback> threadPoolCallbackRunner(ThreadPool &
auto task = std::make_shared<std::packaged_task<Result()>>([thread_group, thread_name, callback = std::move(callback)]() mutable -> Result
{
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
SCOPE_EXIT_SAFE({
{
@ -33,7 +33,7 @@ ThreadPoolCallbackRunner<Result, Callback> threadPoolCallbackRunner(ThreadPool &
}
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
});

View File

@ -36,14 +36,14 @@ static void threadFunction(CompletedPipelineExecutor::Data & data, ThreadGroupSt
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
setThreadName("QueryCompPipeEx");
try
{
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
data.executor->execute(num_threads);
}

View File

@ -308,12 +308,12 @@ void PipelineExecutor::spawnThreads()
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
setThreadName("QueryPipelineEx");
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
try
{

View File

@ -71,14 +71,14 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
setThreadName("QueryPullPipeEx");
try
{
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
data.executor->execute(num_threads);
}

View File

@ -101,14 +101,14 @@ static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGrou
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
setThreadName("QueryPushPipeEx");
try
{
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
data.executor->execute(num_threads);
}

View File

@ -100,11 +100,11 @@ namespace DB
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
setThreadName("Collector");
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
try
{
@ -161,11 +161,11 @@ namespace DB
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
setThreadName("Formatter");
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
try
{

View File

@ -12,10 +12,10 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToGroup(thread_group);
setThreadName("Segmentator");
try
@ -62,10 +62,10 @@ void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr threa
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
const auto parser_unit_number = current_ticket_number % processing_units.size();
auto & unit = processing_units[parser_unit_number];

View File

@ -100,10 +100,10 @@ struct ManyAggregatedData
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
setThreadName("AggregDestruct");
});

View File

@ -286,12 +286,7 @@ Chain buildPushingToViewsChain(
std::unique_ptr<ThreadStatus> view_thread_status_ptr = std::make_unique<ThreadStatus>();
/// Copy of a ThreadStatus should be internal.
view_thread_status_ptr->setInternalThread();
/// view_thread_status_ptr will be moved later (on and on), so need to capture raw pointer.
view_thread_status_ptr->deleter = [thread_status = view_thread_status_ptr.get(), running_group]
{
thread_status->detachQuery();
};
view_thread_status_ptr->attachQuery(running_group);
view_thread_status_ptr->attachToGroup(running_group);
auto * view_thread_status = view_thread_status_ptr.get();
views_data->thread_status_holder->thread_statuses.push_front(std::move(view_thread_status_ptr));

View File

@ -506,6 +506,7 @@ void TCPHandler::runImpl()
/// (i.e. deallocations from the Aggregator with two-level aggregation)
state.reset();
query_scope.reset();
last_sent_snapshots.clear();
thread_trace_context.reset();
}
catch (const Exception & e)

View File

@ -293,12 +293,12 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
setThreadName("DistrOutStrProc");
++job.blocks_started;

View File

@ -1395,10 +1395,10 @@ std::vector<MergeTreeData::LoadPartResult> MergeTreeData::loadDataPartsFromDisk(
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
while (true)
{
@ -2314,10 +2314,10 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
asMutableDeletingPart(part)->remove();
if (part_names_succeed)
@ -2375,10 +2375,10 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
LOG_TRACE(log, "Removing {} parts in blocks range {}", batch.size(), range.getPartNameForLogs());

View File

@ -1119,10 +1119,10 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
CurrentThread::attachToGroupIfDetached(thread_group);
process_part(part_index);
});

View File

@ -90,7 +90,7 @@ namespace
const ucontext_t signal_context = *reinterpret_cast<ucontext_t *>(context);
stack_trace = StackTrace(signal_context);
std::string_view query_id = CurrentThread::getQueryId();
auto query_id = CurrentThread::getQueryId();
query_id_size = std::min(query_id.size(), max_query_id_size);
if (!query_id.empty())
memcpy(query_id_data, query_id.data(), query_id_size);

View File

@ -0,0 +1,23 @@
INSERT TO S3
[ 0 ] S3CompleteMultipartUpload: 1
[ 0 ] S3CreateMultipartUpload: 1
[ 0 ] S3HeadObject: 1
[ 0 ] S3ReadRequestsCount: 1
[ 0 ] S3UploadPart: 1
[ 0 ] S3WriteRequestsCount: 3
CHECK WITH query_log
QueryFinish S3CreateMultipartUpload 1 S3UploadPart 1 S3CompleteMultipartUpload 1 S3PutObject 0
CREATE
INSERT
[ 0 ] FileOpen: 7
READ
INSERT and READ INSERT
[ 0 ] FileOpen: 7
[ 0 ] FileOpen: 7
DROP
CHECK with query_log
QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; FileOpen 7
QueryFinish SELECT \'1\', min(t) FROM times; FileOpen 0
QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; FileOpen 7
QueryFinish SELECT \'2\', min(t) FROM times; FileOpen 0
QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; FileOpen 7

View File

@ -0,0 +1,77 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tag no-fasttest: needs s3
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
echo "INSERT TO S3"
$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq "
INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1;
" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | sort
echo "CHECK WITH query_log"
$CLICKHOUSE_CLIENT -nq "
SYSTEM FLUSH LOGS;
SELECT type,
'S3CreateMultipartUpload', ProfileEvents['S3CreateMultipartUpload'],
'S3UploadPart', ProfileEvents['S3UploadPart'],
'S3CompleteMultipartUpload', ProfileEvents['S3CompleteMultipartUpload'],
'S3PutObject', ProfileEvents['S3PutObject']
FROM system.query_log
WHERE query LIKE '%profile_events.csv%'
AND type = 'QueryFinish'
AND current_database = currentDatabase()
ORDER BY query_start_time DESC;
"
echo "CREATE"
$CLICKHOUSE_CLIENT -nq "
DROP TABLE IF EXISTS times;
CREATE TABLE times (t DateTime) ENGINE MergeTree ORDER BY t
SETTINGS
storage_policy='default',
min_rows_for_compact_part = 0,
min_bytes_for_compact_part = 0,
min_rows_for_wide_part = 1000000,
min_bytes_for_wide_part = 1000000,
in_memory_parts_enable_wal = 0,
ratio_of_defaults_for_sparse_serialization=1.0;
"
echo "INSERT"
$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq "
INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0;
" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ '
echo "READ"
$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq "
SELECT '1', min(t) FROM times;
" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ '
echo "INSERT and READ INSERT"
$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq "
INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0;
SELECT '2', min(t) FROM times;
INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0;
" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ '
echo "DROP"
$CLICKHOUSE_CLIENT -nq "
DROP TABLE times;
"
echo "CHECK with query_log"
$CLICKHOUSE_CLIENT -nq "
SYSTEM FLUSH LOGS;
SELECT type,
query,
'FileOpen', ProfileEvents['FileOpen']
FROM system.query_log
WHERE current_database = currentDatabase()
AND ( query LIKE '%SELECT % FROM times%' OR query LIKE '%INSERT INTO times%' )
AND type = 'QueryFinish'
ORDER BY query_start_time_microseconds ASC, query DESC;
"