ClickHouse/src/Interpreters/ProcessList.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

453 lines
14 KiB
C++
Raw Normal View History

#pragma once
#include <Core/Defines.h>
#include <IO/Progress.h>
#include <Interpreters/CancellationCode.h>
#include <Interpreters/ClientInfo.h>
#include <Interpreters/QueryPriorities.h>
#include <Interpreters/TemporaryDataOnDisk.h>
#include <Interpreters/Context.h>
2021-11-26 11:44:39 +00:00
#include <QueryPipeline/BlockIO.h>
#include <QueryPipeline/ExecutionSpeedLimits.h>
#include <Storages/IStorage_fwd.h>
#include <Poco/Condition.h>
#include <Parsers/IAST.h>
#include <Common/CurrentMetrics.h>
#include <Common/CurrentThread.h>
#include <Common/MemoryTracker.h>
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>
#include <Common/Throttler.h>
2021-10-22 12:56:09 +00:00
#include <Common/OvercommitTracker.h>
#include <condition_variable>
#include <list>
#include <map>
#include <memory>
#include <mutex>
#include <unordered_map>
#include <vector>
namespace DB
{
struct Settings;
2017-01-27 17:27:33 +00:00
class IAST;
class PipelineExecutor;
struct ProcessListForUser;
class QueryStatus;
class ThreadStatus;
class ProcessListEntry;
/** List of currently executing queries.
* Also implements limit on their number.
*/
/** Information of process list element.
* To output in SHOW PROCESSLIST query. Does not contain any complex objects, that do something on copy or destructor.
*/
struct QueryStatusInfo
{
String query;
2023-02-03 01:16:30 +00:00
IAST::QueryKind query_kind{};
UInt64 elapsed_microseconds;
size_t read_rows;
size_t read_bytes;
size_t total_rows;
size_t written_rows;
size_t written_bytes;
Int64 memory_usage;
Int64 peak_memory_usage;
ClientInfo client_info;
bool is_cancelled;
bool is_all_data_sent;
2020-02-02 02:27:15 +00:00
/// Optional fields, filled by query
std::vector<UInt64> thread_ids;
2023-09-07 01:32:39 +00:00
size_t peak_threads_usage;
std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters;
std::shared_ptr<Settings> query_settings;
std::string current_database;
};
/// Query and information about its execution.
class QueryStatus : public WithContext
{
protected:
friend class ProcessList;
friend class ThreadStatus;
2018-05-29 18:14:31 +00:00
friend class CurrentThread;
friend class ProcessListEntry;
friend struct ::GlobalOvercommitTracker;
2015-04-16 06:12:35 +00:00
String query;
ClientInfo client_info;
/// Info about all threads involved in query execution
ThreadGroupPtr thread_group;
2015-04-16 06:12:35 +00:00
Stopwatch watch;
/// Progress of input stream
Progress progress_in;
/// Progress of output stream
Progress progress_out;
2021-11-26 11:44:39 +00:00
/// Used to externally check for the query time limits
/// They are saved in the constructor to limit the overhead of each call to checkTimeLimit()
ExecutionSpeedLimits limits;
OverflowMode overflow_mode;
QueryPriorities::Handle priority_handle = nullptr;
/// True if query cancellation is in progress right now
/// ProcessListEntry should not be destroyed if is_cancelling is true
/// Flag changes is synced with ProcessListBase::mutex and notified with ProcessList::cancelled_cv
bool is_cancelling { false };
/// KILL was send to the query
std::atomic<bool> is_killed { false };
/// All data to the client already had been sent.
/// Including EndOfStream or Exception.
std::atomic<bool> is_all_data_sent { false };
void setUserProcessList(ProcessListForUser * user_process_list_);
/// Be careful using it. For example, queries field of ProcessListForUser could be modified concurrently.
const ProcessListForUser * getUserProcessList() const { return user_process_list; }
2021-09-16 17:40:42 +00:00
mutable std::mutex executors_mutex;
2023-03-02 17:20:29 +00:00
struct ExecutorHolder
{
ExecutorHolder(PipelineExecutor * e) : executor(e) {}
void cancel();
void remove();
PipelineExecutor * executor;
std::mutex mutex;
};
using ExecutorHolderPtr = std::shared_ptr<ExecutorHolder>;
2023-03-27 13:19:45 +00:00
/// Container of PipelineExecutors to be cancelled when a cancelQuery is received
std::unordered_map<PipelineExecutor *, ExecutorHolderPtr> executors;
enum QueryStreamsStatus
{
NotInitialized,
Initialized,
Released
};
QueryStreamsStatus query_streams_status{NotInitialized};
2016-11-30 17:31:05 +00:00
ProcessListForUser * user_process_list = nullptr;
2022-10-21 22:19:06 +00:00
OvercommitTracker * global_overcommit_tracker = nullptr;
2023-02-01 02:11:54 +00:00
/// This is used to control the maximum number of SELECT or INSERT queries.
2023-02-03 01:16:30 +00:00
IAST::QueryKind query_kind{};
2022-02-01 12:06:49 +00:00
/// This field is unused in this class, but it
/// increments/decrements metric in constructor/destructor.
CurrentMetrics::Increment num_queries_increment;
2016-11-30 17:31:05 +00:00
public:
QueryStatus(
ContextPtr context_,
const String & query_,
const ClientInfo & client_info_,
QueryPriorities::Handle && priority_handle_,
ThreadGroupPtr && thread_group_,
2023-02-01 02:31:31 +00:00
IAST::QueryKind query_kind_,
2023-06-23 21:42:31 +00:00
const Settings & query_settings_,
UInt64 watch_start_nanoseconds);
~QueryStatus();
const ClientInfo & getClientInfo() const
{
return client_info;
}
ProgressValues getProgressIn() const
{
return progress_in.getValues();
}
ProgressValues getProgressOut() const
{
return progress_out.getValues();
}
ThrottlerPtr getUserNetworkThrottler();
2021-10-22 12:56:09 +00:00
MemoryTracker * getMemoryTracker() const
{
if (!thread_group)
return nullptr;
2021-10-22 12:56:09 +00:00
return &thread_group->memory_tracker;
}
bool updateProgressIn(const Progress & value)
2015-04-16 06:12:35 +00:00
{
CurrentThread::updateProgressIn(value);
progress_in.incrementPiecewiseAtomically(value);
if (priority_handle)
priority_handle->waitIfNeed(std::chrono::seconds(1)); /// NOTE Could make timeout customizable.
return !is_killed.load(std::memory_order_relaxed);
2015-04-16 06:12:35 +00:00
}
bool updateProgressOut(const Progress & value)
{
CurrentThread::updateProgressOut(value);
progress_out.incrementPiecewiseAtomically(value);
return !is_killed.load(std::memory_order_relaxed);
}
QueryStatusInfo getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const;
CancellationCode cancelQuery(bool kill);
bool isKilled() const { return is_killed; }
bool isAllDataSent() const { return is_all_data_sent; }
void setAllDataSent() { is_all_data_sent = true; }
/// Adds a pipeline to the QueryStatus
void addPipelineExecutor(PipelineExecutor * e);
/// Removes a pipeline to the QueryStatus
void removePipelineExecutor(PipelineExecutor * e);
2021-11-26 11:44:39 +00:00
/// Checks the query time limits (cancelled or timeout)
bool checkTimeLimit();
2021-12-02 13:53:55 +00:00
/// Same as checkTimeLimit but it never throws
[[nodiscard]] bool checkTimeLimitSoft();
/// Get the reference for the start of the query. Used to synchronize with other Stopwatches
UInt64 getQueryCPUStartTime() { return watch.getStart(); }
2015-04-16 06:12:35 +00:00
};
2022-10-17 02:21:08 +00:00
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
2015-04-16 06:12:35 +00:00
/// Information of process list for user.
struct ProcessListForUserInfo
{
Int64 memory_usage;
Int64 peak_memory_usage;
// Optional field, filled by request.
std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters;
};
/// Data about queries for one user.
struct ProcessListForUser
{
2022-09-21 12:51:46 +00:00
explicit ProcessListForUser(ProcessList * global_process_list);
ProcessListForUser(ContextPtr global_context, ProcessList * global_process_list);
2018-03-09 22:11:42 +00:00
/// query_id -> ProcessListElement(s). There can be multiple queries with the same query_id as long as all queries except one are cancelled.
2022-10-17 02:21:08 +00:00
using QueryToElement = std::unordered_map<String, QueryStatusPtr>;
QueryToElement queries;
ProfileEvents::Counters user_performance_counters{VariableContext::User, &ProfileEvents::global_counters};
/// Limit and counter for memory of all simultaneously running queries of single user.
MemoryTracker user_memory_tracker{VariableContext::User};
TemporaryDataOnDiskScopePtr user_temp_data_on_disk;
2021-10-22 12:56:09 +00:00
UserOvercommitTracker user_overcommit_tracker;
2017-08-29 20:20:21 +00:00
/// Count network usage for all simultaneously running queries of single user.
ThrottlerPtr user_throttler;
2018-03-09 22:11:42 +00:00
ProcessListForUserInfo getInfo(bool get_profile_events = false) const;
2018-03-09 22:11:42 +00:00
/// Clears MemoryTracker for the user.
/// Sometimes it is important to reset the MemoryTracker, because it may accumulate skew
/// due to the fact that there are cases when memory can be allocated while processing the query, but released later.
/// Clears network bandwidth Throttler, so it will not count periods of inactivity.
void resetTrackers()
2018-03-09 22:11:42 +00:00
{
/// TODO: should we drop user_temp_data_on_disk here?
2018-03-09 22:11:42 +00:00
user_memory_tracker.reset();
if (user_throttler)
user_throttler.reset();
2018-03-09 22:11:42 +00:00
}
};
class ProcessList;
/// Keeps iterator to process list and removes element in destructor.
class ProcessListEntry
{
private:
2022-10-17 02:21:08 +00:00
using Container = std::list<QueryStatusPtr>;
ProcessList & parent;
Container::iterator it;
public:
ProcessListEntry(ProcessList & parent_, Container::iterator it_)
: parent(parent_), it(it_) {}
~ProcessListEntry();
2022-10-17 02:21:08 +00:00
QueryStatusPtr getQueryStatus() { return *it; }
const QueryStatusPtr getQueryStatus() const { return *it; }
};
2022-08-29 18:24:35 +00:00
class ProcessListBase
{
mutable std::mutex mutex;
protected:
using Lock = std::unique_lock<std::mutex>;
struct LockAndBlocker
{
Lock lock;
OvercommitTrackerBlockerInThread blocker;
};
2022-08-29 18:26:37 +00:00
// It is forbidden to do allocations/deallocations with acquired mutex and
2022-08-29 18:24:35 +00:00
// enabled OvercommitTracker. This leads to deadlock in the case of OOM.
LockAndBlocker safeLock() const noexcept { return { std::unique_lock{mutex}, {} }; }
Lock unsafeLock() const noexcept { return std::unique_lock{mutex}; }
};
class ProcessList : public ProcessListBase
2015-04-16 06:12:35 +00:00
{
public:
2022-10-17 02:21:08 +00:00
using Element = QueryStatusPtr;
using Entry = ProcessListEntry;
using QueryAmount = UInt64;
/// list, for iterators not to invalidate. NOTE: could replace with cyclic buffer, but not worth.
using Container = std::list<Element>;
using Info = std::vector<QueryStatusInfo>;
using UserInfo = std::unordered_map<String, ProcessListForUserInfo>;
/// User -> queries
using UserToQueries = std::unordered_map<String, ProcessListForUser>;
2022-01-10 12:49:53 +00:00
using QueryKindAmounts = std::unordered_map<IAST::QueryKind, QueryAmount>;
protected:
friend class ProcessListEntry;
2022-08-29 18:24:35 +00:00
friend struct ::OvercommitTracker;
2022-02-16 20:02:14 +00:00
friend struct ::UserOvercommitTracker;
friend struct ::GlobalOvercommitTracker;
2018-04-18 20:18:18 +00:00
mutable std::condition_variable have_space; /// Number of currently running queries has become less than maximum.
2014-09-10 11:34:26 +00:00
/// List of queries
Container processes;
/// Notify about cancelled queries (done with ProcessListBase::mutex acquired).
mutable std::condition_variable cancelled_cv;
2021-05-08 22:09:16 +00:00
size_t max_size = 0; /// 0 means no limit. Otherwise, when limit exceeded, an exception is thrown.
/// Stores per-user info: queries, statistics and limits
UserToQueries user_to_queries;
/// Stores info about queries grouped by their priority
QueryPriorities priorities;
/// Limit network bandwidth for all users
ThrottlerPtr total_network_throttler;
/// Call under lock. Finds process with specified current_user and current_query_id.
2022-10-17 02:21:08 +00:00
QueryStatusPtr tryGetProcessListElement(const String & current_query_id, const String & current_user);
/// limit for insert. 0 means no limit. Otherwise, when limit exceeded, an exception is thrown.
size_t max_insert_queries_amount = 0;
/// limit for select. 0 means no limit. Otherwise, when limit exceeded, an exception is thrown.
size_t max_select_queries_amount = 0;
/// amount of queries by query kind.
2022-01-10 12:49:53 +00:00
QueryKindAmounts query_kind_amounts;
2022-01-10 12:49:53 +00:00
void increaseQueryKindAmount(const IAST::QueryKind & query_kind);
void decreaseQueryKindAmount(const IAST::QueryKind & query_kind);
QueryAmount getQueryKindAmount(const IAST::QueryKind & query_kind) const;
public:
using EntryPtr = std::shared_ptr<ProcessListEntry>;
/** Register running query. Returns refcounted object, that will remove element from list in destructor.
2017-11-15 19:47:49 +00:00
* If too many running queries - wait for not more than specified (see settings) amount of time.
* If timeout is passed - throw an exception.
2023-06-19 11:39:30 +00:00
* Don't count KILL QUERY queries or async insert flush queries
*/
2023-06-20 09:37:56 +00:00
EntryPtr insert(const String & query_, const IAST * ast, ContextMutablePtr query_context, UInt64 watch_start_nanoseconds);
/// Number of currently executing queries.
size_t size() const { return processes.size(); }
/// Get current state of process list.
Info getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const;
/// Get current state of process list per user.
UserInfo getUserInfo(bool get_profile_events = false) const;
void setMaxSize(size_t max_size_)
{
2022-08-29 18:24:35 +00:00
auto lock = unsafeLock();
max_size = max_size_;
}
size_t getMaxSize() const
{
auto lock = unsafeLock();
return max_size;
}
void setMaxInsertQueriesAmount(size_t max_insert_queries_amount_)
{
2022-08-29 18:24:35 +00:00
auto lock = unsafeLock();
max_insert_queries_amount = max_insert_queries_amount_;
}
size_t getMaxInsertQueriesAmount() const
{
auto lock = unsafeLock();
return max_insert_queries_amount;
}
void setMaxSelectQueriesAmount(size_t max_select_queries_amount_)
{
2022-08-29 18:24:35 +00:00
auto lock = unsafeLock();
max_select_queries_amount = max_select_queries_amount_;
}
size_t getMaxSelectQueriesAmount() const
{
auto lock = unsafeLock();
return max_select_queries_amount;
}
2016-11-30 17:31:05 +00:00
/// Try call cancel() for input and output streams of query with specified id and user
CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false);
2019-07-30 23:12:04 +00:00
void killAllQueries();
};
}