2013-09-03 20:21:28 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Defines.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <DataStreams/BlockIO.h>
|
2017-11-24 13:55:31 +00:00
|
|
|
#include <IO/Progress.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Interpreters/CancellationCode.h>
|
|
|
|
#include <Interpreters/ClientInfo.h>
|
|
|
|
#include <Interpreters/QueryPriorities.h>
|
|
|
|
#include <Storages/IStorage_fwd.h>
|
|
|
|
#include <Poco/Condition.h>
|
2018-02-01 17:55:08 +00:00
|
|
|
#include <Common/CurrentMetrics.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
|
|
|
#include <Common/MemoryTracker.h>
|
2018-02-01 17:55:08 +00:00
|
|
|
#include <Common/ProfileEvents.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Common/Stopwatch.h>
|
2018-02-01 17:55:08 +00:00
|
|
|
#include <Common/Throttler.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
|
|
|
|
#include <condition_variable>
|
|
|
|
#include <list>
|
|
|
|
#include <map>
|
|
|
|
#include <memory>
|
|
|
|
#include <mutex>
|
|
|
|
#include <shared_mutex>
|
|
|
|
#include <unordered_map>
|
2013-09-03 20:21:28 +00:00
|
|
|
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const Metric Query;
|
2016-10-24 04:06:27 +00:00
|
|
|
}
|
|
|
|
|
2013-09-03 20:21:28 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-01-21 04:24:28 +00:00
|
|
|
struct Settings;
|
2017-01-27 17:27:33 +00:00
|
|
|
class IAST;
|
2018-02-01 17:55:08 +00:00
|
|
|
|
2017-08-29 13:23:04 +00:00
|
|
|
struct ProcessListForUser;
|
2018-05-14 16:09:00 +00:00
|
|
|
class QueryStatus;
|
|
|
|
class ThreadStatus;
|
|
|
|
class ProcessListEntry;
|
2017-01-21 04:24:28 +00:00
|
|
|
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
/** List of currently executing queries.
|
|
|
|
* Also implements limit on their number.
|
2013-09-03 20:21:28 +00:00
|
|
|
*/
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
/** Information of process list element.
|
|
|
|
* To output in SHOW PROCESSLIST query. Does not contain any complex objects, that do something on copy or destructor.
|
2016-01-20 21:32:01 +00:00
|
|
|
*/
|
2018-02-01 17:55:08 +00:00
|
|
|
struct QueryStatusInfo
|
2016-01-20 21:32:01 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
String query;
|
|
|
|
double elapsed_seconds;
|
|
|
|
size_t read_rows;
|
|
|
|
size_t read_bytes;
|
|
|
|
size_t total_rows;
|
|
|
|
size_t written_rows;
|
|
|
|
size_t written_bytes;
|
|
|
|
Int64 memory_usage;
|
2018-03-09 23:04:26 +00:00
|
|
|
Int64 peak_memory_usage;
|
2017-04-01 07:20:54 +00:00
|
|
|
ClientInfo client_info;
|
2018-03-09 21:12:52 +00:00
|
|
|
bool is_cancelled;
|
2018-03-01 16:52:24 +00:00
|
|
|
|
2020-02-02 02:27:15 +00:00
|
|
|
/// Optional fields, filled by query
|
2020-02-02 20:01:13 +00:00
|
|
|
std::vector<UInt64> thread_ids;
|
2018-05-17 16:01:41 +00:00
|
|
|
std::shared_ptr<ProfileEvents::Counters> profile_counters;
|
|
|
|
std::shared_ptr<Settings> query_settings;
|
2021-03-30 13:35:33 +00:00
|
|
|
std::string current_database;
|
2016-01-20 21:32:01 +00:00
|
|
|
};
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
/// Query and information about its execution.
|
2021-04-10 23:33:54 +00:00
|
|
|
class QueryStatus : public WithContext
|
2013-09-03 20:21:28 +00:00
|
|
|
{
|
2018-05-14 16:09:00 +00:00
|
|
|
protected:
|
2018-03-09 23:04:26 +00:00
|
|
|
friend class ProcessList;
|
2018-05-14 16:09:00 +00:00
|
|
|
friend class ThreadStatus;
|
2018-05-29 18:14:31 +00:00
|
|
|
friend class CurrentThread;
|
2018-05-14 16:09:00 +00:00
|
|
|
friend class ProcessListEntry;
|
2018-03-09 23:04:26 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String query;
|
|
|
|
ClientInfo client_info;
|
2018-05-31 15:54:08 +00:00
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
/// Info about all threads involved in query execution
|
|
|
|
ThreadGroupStatusPtr thread_group;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Stopwatch watch;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Progress of input stream
|
|
|
|
Progress progress_in;
|
|
|
|
/// Progress of output stream
|
|
|
|
Progress progress_out;
|
2014-02-12 17:31:02 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
QueryPriorities::Handle priority_handle;
|
|
|
|
|
|
|
|
CurrentMetrics::Increment num_queries_increment{CurrentMetrics::Query};
|
2016-01-20 21:32:01 +00:00
|
|
|
|
2018-04-07 03:49:49 +00:00
|
|
|
std::atomic<bool> is_killed { false };
|
2015-09-04 20:52:00 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
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; }
|
2017-08-29 13:23:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
mutable std::mutex query_streams_mutex;
|
2017-02-03 16:15:12 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Streams with query results, point to BlockIO from executeQuery()
|
|
|
|
/// This declaration is compatible with notes about BlockIO::process_list_entry:
|
|
|
|
/// there are no cyclic dependencies: BlockIO::in,out point to objects inside ProcessListElement (not whole object)
|
|
|
|
BlockInputStreamPtr query_stream_in;
|
|
|
|
BlockOutputStreamPtr query_stream_out;
|
2016-11-30 17:31:05 +00:00
|
|
|
|
2018-04-17 15:16:32 +00:00
|
|
|
enum QueryStreamsStatus
|
|
|
|
{
|
|
|
|
NotInitialized,
|
|
|
|
Initialized,
|
|
|
|
Released
|
|
|
|
};
|
|
|
|
|
|
|
|
QueryStreamsStatus query_streams_status{NotInitialized};
|
2016-11-30 17:31:05 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
ProcessListForUser * user_process_list = nullptr;
|
|
|
|
|
2016-11-30 17:31:05 +00:00
|
|
|
public:
|
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
QueryStatus(
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & query_,
|
|
|
|
const ClientInfo & client_info_,
|
2018-02-01 17:55:08 +00:00
|
|
|
QueryPriorities::Handle && priority_handle_);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-05-31 15:54:08 +00:00
|
|
|
~QueryStatus();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-09 23:04:26 +00:00
|
|
|
const ClientInfo & getClientInfo() const
|
|
|
|
{
|
|
|
|
return client_info;
|
|
|
|
}
|
|
|
|
|
|
|
|
ProgressValues getProgressIn() const
|
|
|
|
{
|
|
|
|
return progress_in.getValues();
|
|
|
|
}
|
|
|
|
|
|
|
|
ProgressValues getProgressOut() const
|
|
|
|
{
|
|
|
|
return progress_out.getValues();
|
|
|
|
}
|
|
|
|
|
|
|
|
ThrottlerPtr getUserNetworkThrottler();
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
bool updateProgressIn(const Progress & value)
|
|
|
|
{
|
2018-05-31 15:54:08 +00:00
|
|
|
CurrentThread::updateProgressIn(value);
|
2017-04-01 07:20:54 +00:00
|
|
|
progress_in.incrementPiecewiseAtomically(value);
|
|
|
|
|
|
|
|
if (priority_handle)
|
|
|
|
priority_handle->waitIfNeed(std::chrono::seconds(1)); /// NOTE Could make timeout customizable.
|
|
|
|
|
2018-04-07 03:49:49 +00:00
|
|
|
return !is_killed.load(std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool updateProgressOut(const Progress & value)
|
|
|
|
{
|
2018-05-31 15:54:08 +00:00
|
|
|
CurrentThread::updateProgressOut(value);
|
2017-04-01 07:20:54 +00:00
|
|
|
progress_out.incrementPiecewiseAtomically(value);
|
2018-05-31 15:54:08 +00:00
|
|
|
|
2018-04-07 03:49:49 +00:00
|
|
|
return !is_killed.load(std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-05-17 16:01:41 +00:00
|
|
|
QueryStatusInfo getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Copies pointers to in/out streams
|
|
|
|
void setQueryStreams(const BlockIO & io);
|
|
|
|
|
|
|
|
/// Frees in/out streams
|
|
|
|
void releaseQueryStreams();
|
|
|
|
|
|
|
|
/// It means that ProcessListEntry still exists, but stream was already destroyed
|
|
|
|
bool streamsAreReleased();
|
|
|
|
|
|
|
|
/// Get query in/out pointers from BlockIO
|
|
|
|
bool tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const;
|
2018-11-01 14:56:37 +00:00
|
|
|
|
2019-02-01 01:48:25 +00:00
|
|
|
CancellationCode cancelQuery(bool kill);
|
|
|
|
|
2018-11-01 14:56:37 +00:00
|
|
|
bool isKilled() const { return is_killed; }
|
2015-04-16 06:12:35 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2020-03-20 17:27:17 +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> profile_counters;
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
/// Data about queries for one user.
|
2015-12-30 15:39:11 +00:00
|
|
|
struct ProcessListForUser
|
|
|
|
{
|
2018-02-01 17:55:08 +00:00
|
|
|
ProcessListForUser();
|
|
|
|
|
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.
|
2019-06-30 13:17:27 +00:00
|
|
|
using QueryToElement = std::unordered_map<String, QueryStatus *>;
|
2017-04-01 07:20:54 +00:00
|
|
|
QueryToElement queries;
|
2016-01-13 02:38:30 +00:00
|
|
|
|
2018-06-09 15:29:08 +00:00
|
|
|
ProfileEvents::Counters user_performance_counters{VariableContext::User, &ProfileEvents::global_counters};
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Limit and counter for memory of all simultaneously running queries of single user.
|
2018-06-09 15:29:08 +00:00
|
|
|
MemoryTracker user_memory_tracker{VariableContext::User};
|
2017-08-29 13:23:04 +00:00
|
|
|
|
2017-08-29 20:20:21 +00:00
|
|
|
/// Count network usage for all simultaneously running queries of single user.
|
2017-08-29 13:23:04 +00:00
|
|
|
ThrottlerPtr user_throttler;
|
2018-03-09 22:11:42 +00:00
|
|
|
|
2020-03-20 17:27:17 +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.
|
2018-06-19 20:30:35 +00:00
|
|
|
void resetTrackers()
|
2018-03-09 22:11:42 +00:00
|
|
|
{
|
|
|
|
user_memory_tracker.reset();
|
|
|
|
if (user_throttler)
|
2018-03-29 13:24:36 +00:00
|
|
|
user_throttler.reset();
|
2018-03-09 22:11:42 +00:00
|
|
|
}
|
2015-12-30 15:39:11 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2015-06-21 06:06:04 +00:00
|
|
|
class ProcessList;
|
|
|
|
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
/// Keeps iterator to process list and removes element in destructor.
|
2015-06-21 06:06:04 +00:00
|
|
|
class ProcessListEntry
|
|
|
|
{
|
|
|
|
private:
|
2018-02-01 17:55:08 +00:00
|
|
|
using Container = std::list<QueryStatus>;
|
2015-06-21 06:06:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ProcessList & parent;
|
|
|
|
Container::iterator it;
|
2018-05-14 16:09:00 +00:00
|
|
|
|
2015-06-21 06:06:04 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
ProcessListEntry(ProcessList & parent_, Container::iterator it_)
|
|
|
|
: parent(parent_), it(it_) {}
|
2015-06-21 06:06:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
~ProcessListEntry();
|
2015-06-21 06:06:04 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
QueryStatus * operator->() { return &*it; }
|
|
|
|
const QueryStatus * operator->() const { return &*it; }
|
2015-06-21 06:06:04 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
QueryStatus & get() { return *it; }
|
|
|
|
const QueryStatus & get() const { return *it; }
|
2015-06-21 06:06:04 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2015-04-16 06:12:35 +00:00
|
|
|
class ProcessList
|
|
|
|
{
|
|
|
|
public:
|
2018-02-01 17:55:08 +00:00
|
|
|
using Element = QueryStatus;
|
2017-04-01 07:20:54 +00:00
|
|
|
using Entry = ProcessListEntry;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// list, for iterators not to invalidate. NOTE: could replace with cyclic buffer, but not worth.
|
|
|
|
using Container = std::list<Element>;
|
2018-02-01 17:55:08 +00:00
|
|
|
using Info = std::vector<QueryStatusInfo>;
|
2020-03-20 17:27:17 +00:00
|
|
|
using UserInfo = std::unordered_map<String, ProcessListForUserInfo>;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// User -> queries
|
|
|
|
using UserToQueries = std::unordered_map<String, ProcessListForUser>;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2018-05-14 16:09:00 +00:00
|
|
|
protected:
|
|
|
|
friend class ProcessListEntry;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
mutable std::mutex mutex;
|
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
|
|
|
|
2017-08-29 13:23:04 +00:00
|
|
|
/// List of queries
|
2018-02-01 17:55:08 +00:00
|
|
|
Container processes;
|
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.
|
2017-08-29 13:23:04 +00:00
|
|
|
|
|
|
|
/// Stores per-user info: queries, statistics and limits
|
2017-04-01 07:20:54 +00:00
|
|
|
UserToQueries user_to_queries;
|
2017-08-29 13:23:04 +00:00
|
|
|
|
|
|
|
/// Stores info about queries grouped by their priority
|
2017-04-01 07:20:54 +00:00
|
|
|
QueryPriorities priorities;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2018-03-29 13:24:36 +00:00
|
|
|
/// Limit network bandwidth for all users
|
|
|
|
ThrottlerPtr total_network_throttler;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Call under lock. Finds process with specified current_user and current_query_id.
|
2018-02-01 17:55:08 +00:00
|
|
|
QueryStatus * tryGetProcessListElement(const String & current_query_id, const String & current_user);
|
2017-01-24 15:11:36 +00:00
|
|
|
|
2013-09-03 20:21:28 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
using EntryPtr = std::shared_ptr<ProcessListEntry>;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** 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.
|
2017-04-01 07:20:54 +00:00
|
|
|
* If timeout is passed - throw an exception.
|
|
|
|
* Don't count KILL QUERY queries.
|
|
|
|
*/
|
2021-04-10 23:33:54 +00:00
|
|
|
EntryPtr insert(const String & query_, const IAST * ast, ContextPtr query_context);
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Number of currently executing queries.
|
2018-02-01 17:55:08 +00:00
|
|
|
size_t size() const { return processes.size(); }
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Get current state of process list.
|
2018-05-17 16:01:41 +00:00
|
|
|
Info getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2020-03-20 17:27:17 +00:00
|
|
|
/// Get current state of process list per user.
|
|
|
|
UserInfo getUserInfo(bool get_profile_events = false) const;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void setMaxSize(size_t max_size_)
|
|
|
|
{
|
2019-01-02 06:44:36 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
max_size = max_size_;
|
|
|
|
}
|
2015-09-04 20:52:00 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Try call cancel() for input and output streams of query with specified id and user
|
2018-03-05 21:09:39 +00:00
|
|
|
CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false);
|
2019-07-30 23:12:04 +00:00
|
|
|
|
|
|
|
void killAllQueries();
|
2013-09-03 20:21:28 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|