ClickHouse/src/Interpreters/ProcessList.cpp

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

638 lines
24 KiB
C++
Raw Normal View History

#include <Interpreters/ProcessList.h>
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
2021-11-01 13:19:31 +00:00
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTKillQueryQuery.h>
2022-01-10 17:23:17 +00:00
#include <Parsers/IAST.h>
#include <Parsers/queryNormalization.h>
#include <Parsers/toOneLineQuery.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Common/typeid_cast.h>
#include <Common/Exception.h>
2018-05-29 18:14:31 +00:00
#include <Common/CurrentThread.h>
#include <IO/WriteHelpers.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
2018-04-18 20:18:18 +00:00
#include <chrono>
2022-02-01 12:06:49 +00:00
namespace CurrentMetrics
{
extern const Metric Query;
}
namespace DB
{
namespace ErrorCodes
{
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_SIMULTANEOUS_QUERIES;
extern const int QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING;
2018-03-09 22:11:42 +00:00
extern const int LOGICAL_ERROR;
2021-11-26 11:44:39 +00:00
extern const int QUERY_WAS_CANCELLED;
}
/// Should we execute the query even if max_concurrent_queries limit is exhausted
static bool isUnlimitedQuery(const IAST * ast)
{
if (!ast)
return false;
/// It is KILL QUERY
2019-03-11 13:22:51 +00:00
if (ast->as<ASTKillQueryQuery>())
return true;
/// It is SELECT FROM system.processes
2018-04-18 21:14:47 +00:00
/// NOTE: This is very rough check.
/// False negative: USE system; SELECT * FROM processes;
/// False positive: SELECT * FROM system.processes CROSS JOIN (SELECT ...)
2019-03-11 13:22:51 +00:00
if (const auto * ast_selects = ast->as<ASTSelectWithUnionQuery>())
{
if (!ast_selects->list_of_selects || ast_selects->list_of_selects->children.empty())
return false;
2019-03-11 13:22:51 +00:00
const auto * ast_select = ast_selects->list_of_selects->children[0]->as<ASTSelectQuery>();
if (!ast_select)
return false;
if (auto database_and_table = getDatabaseAndTable(*ast_select, 0))
return database_and_table->database == "system" && database_and_table->table == "processes";
return false;
}
return false;
}
2022-09-01 12:22:49 +00:00
ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr query_context)
{
EntryPtr res;
const ClientInfo & client_info = query_context->getClientInfo();
const Settings & settings = query_context->getSettingsRef();
if (client_info.current_query_id.empty())
throw Exception("Query id cannot be empty", ErrorCodes::LOGICAL_ERROR);
bool is_unlimited_query = isUnlimitedQuery(ast);
{
2022-08-29 18:24:35 +00:00
auto [lock, overcommit_blocker] = safeLock(); // To avoid deadlock in case of OOM
2022-01-10 17:23:17 +00:00
IAST::QueryKind query_kind = ast->getQueryKind();
const auto queue_max_wait_ms = settings.queue_max_wait_ms.totalMilliseconds();
if (!is_unlimited_query && max_size && processes.size() >= max_size)
{
if (queue_max_wait_ms)
2020-05-30 21:57:37 +00:00
LOG_WARNING(&Poco::Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms);
if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), [&]{ return processes.size() < max_size; }))
throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES);
}
if (!is_unlimited_query)
{
2022-01-10 17:23:17 +00:00
QueryAmount amount = getQueryKindAmount(query_kind);
if (max_insert_queries_amount && query_kind == IAST::QueryKind::Insert && amount >= max_insert_queries_amount)
throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES,
"Too many simultaneous insert queries. Maximum: {}, current: {}",
max_insert_queries_amount, amount);
2022-01-10 17:23:17 +00:00
if (max_select_queries_amount && query_kind == IAST::QueryKind::Select && amount >= max_select_queries_amount)
throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES,
"Too many simultaneous select queries. Maximum: {}, current: {}",
max_select_queries_amount, amount);
}
{
/**
* `max_size` check above is controlled by `max_concurrent_queries` server setting and is a "hard" limit for how many
* queries the server can process concurrently. It is configured at startup. When the server is overloaded with queries and the
* hard limit is reached it is impossible to connect to the server to run queries for investigation.
*
* With `max_concurrent_queries_for_all_users` it is possible to configure an additional, runtime configurable, limit for query concurrency.
* Usually it should be configured just once for `default_profile` which is inherited by all users. DBAs can override
* this setting when connecting to ClickHouse, or it can be configured for a DBA profile to have a value greater than that of
* the default profile (or 0 for unlimited).
*
* One example is to set `max_size=X`, `max_concurrent_queries_for_all_users=X-10` for default profile,
* and `max_concurrent_queries_for_all_users=0` for DBAs or accounts that are vital for ClickHouse operations (like metrics
* exporters).
*
* Another creative example is to configure `max_concurrent_queries_for_all_users=50` for "analyst" profiles running adhoc queries
* and `max_concurrent_queries_for_all_users=100` for "customer facing" services. This way "analyst" queries will be rejected
* once is already processing 50+ concurrent queries (including analysts or any other users).
*/
if (!is_unlimited_query && settings.max_concurrent_queries_for_all_users
&& processes.size() >= settings.max_concurrent_queries_for_all_users)
throw Exception(
"Too many simultaneous queries for all users. Current: " + toString(processes.size())
+ ", maximum: " + settings.max_concurrent_queries_for_all_users.toString(),
ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES);
}
2016-10-25 05:07:29 +00:00
/** Why we use current user?
* Because initial one is passed by client and credentials for it is not verified,
* and using initial_user for limits will be insecure.
*
2016-10-25 05:07:29 +00:00
* Why we use current_query_id?
* Because we want to allow distributed queries that will run multiple secondary queries on same server,
* like SELECT count() FROM remote('127.0.0.{1,2}', system.numbers)
* so they must have different query_ids.
*/
{
2016-11-30 17:31:05 +00:00
auto user_process_list = user_to_queries.find(client_info.current_user);
if (user_process_list != user_to_queries.end())
{
if (!is_unlimited_query && settings.max_concurrent_queries_for_user
2016-11-30 17:31:05 +00:00
&& user_process_list->second.queries.size() >= settings.max_concurrent_queries_for_user)
throw Exception("Too many simultaneous queries for user " + client_info.current_user
+ ". Current: " + toString(user_process_list->second.queries.size())
2017-03-14 00:14:19 +00:00
+ ", maximum: " + settings.max_concurrent_queries_for_user.toString(),
2018-03-09 23:23:15 +00:00
ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES);
auto running_query = user_process_list->second.queries.find(client_info.current_query_id);
if (running_query != user_process_list->second.queries.end())
{
if (!settings.replace_running_query)
throw Exception("Query with id = " + client_info.current_query_id + " is already running.",
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
/// Ask queries to cancel. They will check this flag.
running_query->second->is_killed.store(true, std::memory_order_relaxed);
const auto replace_running_query_max_wait_ms = settings.replace_running_query_max_wait_ms.totalMilliseconds();
if (!replace_running_query_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(replace_running_query_max_wait_ms),
[&]
{
running_query = user_process_list->second.queries.find(client_info.current_query_id);
if (running_query == user_process_list->second.queries.end())
return true;
running_query->second->is_killed.store(true, std::memory_order_relaxed);
return false;
}))
{
throw Exception("Query with id = " + client_info.current_query_id + " is already running and can't be stopped",
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
}
}
}
}
/// Check other users running query with our query_id
for (const auto & user_process_list : user_to_queries)
{
if (user_process_list.first == client_info.current_user)
continue;
if (auto running_query = user_process_list.second.queries.find(client_info.current_query_id); running_query != user_process_list.second.queries.end())
throw Exception("Query with id = " + client_info.current_query_id + " is already running by user " + user_process_list.first,
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
}
2022-02-16 20:02:14 +00:00
auto user_process_list_it = user_to_queries.find(client_info.current_user);
if (user_process_list_it == user_to_queries.end())
2022-09-01 12:22:49 +00:00
{
user_process_list_it = user_to_queries.emplace(std::piecewise_construct,
std::forward_as_tuple(client_info.current_user),
std::forward_as_tuple(query_context->getGlobalContext(), this)).first;
}
2022-02-16 20:02:14 +00:00
ProcessListForUser & user_process_list = user_process_list_it->second;
2020-05-01 19:01:02 +00:00
/// Actualize thread group info
auto thread_group = CurrentThread::getGroup();
if (thread_group)
2020-05-01 19:01:02 +00:00
{
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)
{
query_context->setTempDataOnDisk(std::make_shared<TemporaryDataOnDisk>(
user_process_list.user_temp_data_on_disk, settings.max_temp_data_on_disk_size_for_query));
}
thread_group->query = query_;
thread_group->one_line_query = toOneLineQuery(query_);
thread_group->normalized_query_hash = normalizedQueryHash<false>(query_);
2020-05-01 19:01:02 +00:00
/// Set query-level memory trackers
thread_group->memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage);
thread_group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator);
if (query_context->hasTraceCollector())
2020-05-01 19:01:02 +00:00
{
/// Set up memory profiling
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
2020-05-01 19:01:02 +00:00
}
2020-05-01 19:01:02 +00:00
thread_group->memory_tracker.setDescription("(for query)");
2022-09-10 02:07:51 +00:00
if (settings.memory_tracker_fault_probability > 0.0)
2020-05-01 19:01:02 +00:00
thread_group->memory_tracker.setFaultProbability(settings.memory_tracker_fault_probability);
thread_group->memory_tracker.setOvercommitWaitingTime(settings.memory_usage_overcommit_max_wait_microseconds);
2020-05-01 19:01:02 +00:00
/// NOTE: Do not set the limit for thread-level memory tracker since it could show unreal values
/// since allocation and deallocation could happen in different threads
}
2022-02-16 20:02:14 +00:00
auto process_it = processes.emplace(processes.end(),
query_context, query_, client_info, priorities.insert(settings.priority), std::move(thread_group), query_kind);
increaseQueryKindAmount(query_kind);
res = std::make_shared<Entry>(*this, process_it);
process_it->setUserProcessList(&user_process_list);
2022-02-16 20:02:14 +00:00
user_process_list.queries.emplace(client_info.current_query_id, &res->get());
/// Track memory usage for all simultaneously running queries from single user.
user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator_for_user);
user_process_list.user_memory_tracker.setDescription("(for user)");
2020-05-01 19:01:02 +00:00
if (!user_process_list.user_throttler)
{
if (settings.max_network_bandwidth_for_user)
user_process_list.user_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_user, total_network_throttler);
else if (settings.max_network_bandwidth_for_all_users)
user_process_list.user_throttler = total_network_throttler;
}
if (!total_network_throttler && settings.max_network_bandwidth_for_all_users)
{
total_network_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_all_users);
}
}
return res;
}
ProcessListEntry::~ProcessListEntry()
{
2022-08-29 17:44:05 +00:00
auto lock = parent.safeLock();
String user = it->getClientInfo().current_user;
String query_id = it->getClientInfo().current_query_id;
IAST::QueryKind query_kind = it->query_kind;
const QueryStatus * process_list_element_ptr = &*it;
2018-03-09 22:11:42 +00:00
auto user_process_list_it = parent.user_to_queries.find(user);
if (user_process_list_it == parent.user_to_queries.end())
{
2020-05-30 21:57:37 +00:00
LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList");
std::terminate();
}
2018-03-09 22:11:42 +00:00
ProcessListForUser & user_process_list = user_process_list_it->second;
2018-03-09 22:11:42 +00:00
bool found = false;
if (auto running_query = user_process_list.queries.find(query_id); running_query != user_process_list.queries.end())
{
if (running_query->second == process_list_element_ptr)
{
user_process_list.queries.erase(running_query->first);
found = true;
}
}
/// Wait for the query if it is in the cancellation right now.
parent.cancelled_cv.wait(lock.lock, [&]() { return it->is_cancelling == false; });
2022-02-16 20:02:14 +00:00
/// This removes the memory_tracker of one request.
parent.processes.erase(it);
if (!found)
{
2020-05-30 21:57:37 +00:00
LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
std::terminate();
}
parent.decreaseQueryKindAmount(query_kind);
parent.have_space.notify_all();
/// If there are no more queries for the user, then we will reset memory tracker and network throttler.
if (user_process_list.queries.empty())
user_process_list.resetTrackers();
2020-04-19 21:43:06 +00:00
/// Reset throttler, similarly (see above).
2020-03-08 21:40:00 +00:00
if (parent.processes.empty())
parent.total_network_throttler.reset();
}
QueryStatus::QueryStatus(
ContextPtr context_,
const String & query_,
const ClientInfo & client_info_,
QueryPriorities::Handle && priority_handle_,
ThreadGroupStatusPtr && thread_group_,
IAST::QueryKind query_kind_)
: WithContext(context_)
, query(query_)
, client_info(client_info_)
, thread_group(std::move(thread_group_))
, priority_handle(std::move(priority_handle_))
, query_kind(query_kind_)
2022-02-01 12:06:49 +00:00
, num_queries_increment(CurrentMetrics::Query)
{
2021-11-26 11:44:39 +00:00
auto settings = getContext()->getSettings();
limits.max_execution_time = settings.max_execution_time;
overflow_mode = settings.timeout_overflow_mode;
}
QueryStatus::~QueryStatus()
{
assert(executors.empty());
if (auto * memory_tracker = getMemoryTracker())
{
if (user_process_list)
user_process_list->user_overcommit_tracker.onQueryStop(memory_tracker);
if (auto shared_context = getContext())
shared_context->getGlobalOvercommitTracker()->onQueryStop(memory_tracker);
}
}
2021-09-16 17:40:42 +00:00
CancellationCode QueryStatus::cancelQuery(bool)
{
2021-09-16 17:40:42 +00:00
if (is_killed.load())
return CancellationCode::CancelSent;
2021-09-21 08:01:00 +00:00
is_killed.store(true);
2021-09-20 19:07:33 +00:00
std::lock_guard lock(executors_mutex);
for (auto * e : executors)
e->cancel();
return CancellationCode::CancelSent;
}
void QueryStatus::addPipelineExecutor(PipelineExecutor * e)
{
Fix possible deadlock with async_socket_for_remote/use_hedged_requests and parallel KILL Right now it is possible to call QueryStatus::addPipelineExecutor() when the executors_mutex already acquired, it is possible when the query was cancelled via KILL QUERY. Here I will show some traces from debugger from a real example, where tons of ProcessList::insert() got deadlocked. Let's look at the lock owner for one of the threads that was deadlocked in ProcessList::insert(): (gdb) p *mutex $2 = { __data = { __owner = 46899, }, } And now let's see the stack trace of the 46899: #0 __lll_lock_wait () at ../sysdeps/unix/sysv/linux/x86_64/lowlevellock.S:103 #1 0x00007fb65569b714 in __GI___pthread_mutex_lock (mutex=0x7fb4a9d15298) at ../nptl/pthread_mutex_lock.c:80 #2 0x000000001b6edd91 in pthread_mutex_lock (arg=0x7fb4a9d15298) at ../src/Common/ThreadFuzzer.cpp:317 #3 std::__1::__libcpp_mutex_lock (__m=0x7fb4a9d15298) at ../contrib/libcxx/include/__threading_support:303 #4 std::__1::mutex::lock (this=0x7fb4a9d15298) at ../contrib/libcxx/src/mutex.cpp:33 #5 0x0000000014c7ae63 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91 #6 DB::QueryStatus::addPipelineExecutor (this=0x7fb4a9d14f90, e=0x80) at ../src/Interpreters/ProcessList.cpp:372 #7 0x0000000015bee4a7 in DB::PipelineExecutor::PipelineExecutor (this=0x7fb4b1e53618, processors=..., elem=<optimized out>) at ../src/Processors/Executors/PipelineExecutor.cpp:54 #12 std::__1::make_shared<DB::PipelineExecutor, std::__1::vector<std::__1::shared_ptr<DB::IProcessor>, std::__1::allocator<std::__1::shared_ptr<DB::IProcessor> > >&, DB::QueryStatus*&, void> (__args=@0x7fb63095b9b0: 0x7fb4a9d14f90, __args=@0x7fb63095b9b0: 0x7fb4a9d14f90) at ../contrib/libcxx/include/__memory/shared_ptr.h:963 #13 DB::QueryPipelineBuilder::execute (this=0x7fb63095b8b0) at ../src/QueryPipeline/QueryPipelineBuilder.cpp:552 #14 0x00000000158c6c27 in DB::Connection::sendExternalTablesData (this=0x7fb6545e9d98, data=...) at ../src/Client/Connection.cpp:797 #27 0x0000000014043a81 in DB::RemoteQueryExecutorRoutine::operator() (this=0x7fb63095bf20, sink=...) at ../src/QueryPipeline/RemoteQueryExecutorReadContext.cpp:46 #32 0x000000000a16dd4f in make_fcontext () at ../contrib/boost/libs/context/src/asm/make_x86_64_sysv_elf_gas.S:71 And also in the logs you can see very strange things for this thread: 2022.09.13 14:14:51.228979 [ 51145 ] {1712D4E914EC7C99} <Debug> Connection (localhost:9000): Sent data for 1 external tables, total 11 rows in 0.00046389 sec., 23688 rows/sec., 3.84 KiB (8.07 MiB/sec.), compressed 1.1070121092649958 times to 3.47 KiB (7.29 MiB/sec.) ... 2022.09.13 14:14:51.719402 [ 46899 ] {7c90ffa4-1dc8-42fd-938c-4e307c244394} <Debug> executeQuery: (from 10.101.15.181:42478) KILL QUERY WHERE query_id = '1712D4E914EC7C99' (stage: Complete) 2022.09.13 14:14:51.719488 [ 46899 ] {7c90ffa4-1dc8-42fd-938c-4e307c244394} <Debug> executeQuery: (internal) SELECT query_id, user, query FROM system.processes WHERE query_id = '1712D4E914EC7C99' (stage: Complete) 2022.09.13 14:14:51.719754 [ 46899 ] {7c90ffa4-1dc8-42fd-938c-4e307c244394} <Trace> ContextAccess (default): Access granted: SELECT(user, query_id, query) ON system.processes 2022.09.13 14:14:51.720544 [ 46899 ] {7c90ffa4-1dc8-42fd-938c-4e307c244394} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2022.09.13 14:14:53.228964 [ 46899 ] {7c90ffa4-1dc8-42fd-938c-4e307c244394} <Debug> Connection (localhost:9000): Sent data for 2 scalars, total 2 rows in 2.6838e-05 sec., 73461 rows/sec., 68.00 B (2.38 MiB/sec.), compressed 0.4594594594594595 times to 148.00 B (5.16 MiB/sec.) How is this possible? The answer is fibers and query cancellation routine. During cancellation of async queries it going into fibers again and try to do this gracefully. However because of this during canceling query it may call QueryStatus::addPipelineExecutor() from QueryStatus::cancelQuery(). Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-09-15 09:15:19 +00:00
/// In case of asynchronous distributed queries it is possible to call
/// addPipelineExecutor() from the cancelQuery() context, and this will
/// lead to deadlock.
if (is_killed.load())
throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED);
2021-09-16 17:40:42 +00:00
std::lock_guard lock(executors_mutex);
assert(std::find(executors.begin(), executors.end(), e) == executors.end());
executors.push_back(e);
}
void QueryStatus::removePipelineExecutor(PipelineExecutor * e)
{
2021-09-16 17:40:42 +00:00
std::lock_guard lock(executors_mutex);
assert(std::find(executors.begin(), executors.end(), e) != executors.end());
std::erase_if(executors, [e](PipelineExecutor * x) { return x == e; });
}
2021-11-26 11:44:39 +00:00
bool QueryStatus::checkTimeLimit()
{
if (is_killed.load())
throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED);
return limits.checkTimeLimit(watch, overflow_mode);
}
2021-12-02 13:53:55 +00:00
bool QueryStatus::checkTimeLimitSoft()
{
if (is_killed.load())
return false;
return limits.checkTimeLimit(watch, OverflowMode::BREAK);
}
2016-11-30 17:31:05 +00:00
void QueryStatus::setUserProcessList(ProcessListForUser * user_process_list_)
{
user_process_list = user_process_list_;
}
ThrottlerPtr QueryStatus::getUserNetworkThrottler()
{
if (!user_process_list)
return {};
return user_process_list->user_throttler;
}
QueryStatus * ProcessList::tryGetProcessListElement(const String & current_query_id, const String & current_user)
{
auto user_it = user_to_queries.find(current_user);
if (user_it != user_to_queries.end())
{
const auto & user_queries = user_it->second.queries;
auto query_it = user_queries.find(current_query_id);
if (query_it != user_queries.end())
return query_it->second;
}
return nullptr;
}
CancellationCode ProcessList::sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill)
2016-11-30 17:31:05 +00:00
{
QueryStatus * elem;
/// Cancelling the query should be done without the lock.
///
/// Since it may be not that trivial, for example in case of distributed
/// queries it tries to cancel the query gracefully on shards and this can
/// take a while, so acquiring a lock during this time will lead to wait
/// all new queries for this cancellation.
///
/// Another problem is that it can lead to a deadlock, because of
/// OvercommitTracker.
///
/// So here we first set is_cancelling, and later reset it.
/// The ProcessListEntry cannot be destroy if is_cancelling is true.
{
auto lock = safeLock();
elem = tryGetProcessListElement(current_query_id, current_user);
if (!elem)
return CancellationCode::NotFound;
elem->is_cancelling = true;
}
2016-11-30 17:31:05 +00:00
SCOPE_EXIT({
DENY_ALLOCATIONS_IN_SCOPE;
auto lock = unsafeLock();
elem->is_cancelling = false;
cancelled_cv.notify_all();
});
return elem->cancelQuery(kill);
2016-11-30 17:31:05 +00:00
}
2019-07-30 23:12:04 +00:00
void ProcessList::killAllQueries()
{
std::vector<QueryStatus *> cancelled_processes;
SCOPE_EXIT({
auto lock = safeLock();
for (auto & cancelled_process : cancelled_processes)
cancelled_process->is_cancelling = false;
cancelled_cv.notify_all();
});
{
auto lock = safeLock();
cancelled_processes.reserve(processes.size());
for (auto & process : processes)
{
cancelled_processes.push_back(&process);
process.is_cancelling = true;
}
}
for (auto & cancelled_process : cancelled_processes)
cancelled_process->cancelQuery(true);
2019-07-30 23:12:04 +00:00
}
QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const
{
2020-09-24 23:24:58 +00:00
QueryStatusInfo res{};
res.query = query;
res.client_info = client_info;
res.elapsed_seconds = watch.elapsedSeconds();
res.is_cancelled = is_killed.load(std::memory_order_relaxed);
res.is_all_data_sent = is_all_data_sent.load(std::memory_order_relaxed);
res.read_rows = progress_in.read_rows;
res.read_bytes = progress_in.read_bytes;
res.total_rows = progress_in.total_rows_to_read;
2019-05-21 04:06:36 +00:00
res.written_rows = progress_out.written_rows;
res.written_bytes = progress_out.written_bytes;
if (thread_group)
{
res.memory_usage = thread_group->memory_tracker.get();
res.peak_memory_usage = thread_group->memory_tracker.getPeak();
if (get_thread_list)
{
std::lock_guard lock(thread_group->mutex);
2020-02-02 02:27:15 +00:00
res.thread_ids = thread_group->thread_ids;
}
if (get_profile_events)
res.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(thread_group->performance_counters.getPartiallyAtomicSnapshot());
}
if (get_settings && getContext())
{
res.query_settings = std::make_shared<Settings>(getContext()->getSettings());
res.current_database = getContext()->getCurrentDatabase();
}
return res;
}
ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const
{
Info per_query_infos;
2022-08-29 17:44:05 +00:00
auto lock = safeLock();
per_query_infos.reserve(processes.size());
for (const auto & process : processes)
per_query_infos.emplace_back(process.getInfo(get_thread_list, get_profile_events, get_settings));
return per_query_infos;
}
ProcessListForUser::ProcessListForUser(ContextPtr global_context, ProcessList * global_process_list)
2022-02-16 20:02:14 +00:00
: user_overcommit_tracker(global_process_list, this)
2021-10-22 12:56:09 +00:00
{
user_memory_tracker.setOvercommitTracker(&user_overcommit_tracker);
if (global_context)
{
size_t size_limit = global_context->getSettingsRef().max_temp_data_on_disk_size_for_user;
2022-09-01 12:22:49 +00:00
user_temp_data_on_disk = std::make_shared<TemporaryDataOnDisk>(global_context->getSharedTempDataOnDisk(), size_limit);
}
2021-10-22 12:56:09 +00:00
}
ProcessListForUserInfo ProcessListForUser::getInfo(bool get_profile_events) const
{
ProcessListForUserInfo res;
res.memory_usage = user_memory_tracker.get();
res.peak_memory_usage = user_memory_tracker.getPeak();
if (get_profile_events)
res.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(user_performance_counters.getPartiallyAtomicSnapshot());
return res;
}
ProcessList::UserInfo ProcessList::getUserInfo(bool get_profile_events) const
{
UserInfo per_user_infos;
2022-08-29 17:44:05 +00:00
auto lock = safeLock();
per_user_infos.reserve(user_to_queries.size());
for (const auto & [user, user_queries] : user_to_queries)
per_user_infos.emplace(user, user_queries.getInfo(get_profile_events));
return per_user_infos;
}
2022-01-10 12:49:53 +00:00
void ProcessList::increaseQueryKindAmount(const IAST::QueryKind & query_kind)
{
2022-01-10 12:49:53 +00:00
auto found = query_kind_amounts.find(query_kind);
if (found == query_kind_amounts.end())
query_kind_amounts[query_kind] = 1;
else
found->second += 1;
}
2022-01-10 12:49:53 +00:00
void ProcessList::decreaseQueryKindAmount(const IAST::QueryKind & query_kind)
{
2022-01-10 12:49:53 +00:00
auto found = query_kind_amounts.find(query_kind);
/// TODO: we could just rebuild the map, as we have saved all query_kind.
if (found == query_kind_amounts.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease before increase on '{}'", query_kind);
else if (found->second == 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease to negative on '{}'", query_kind, found->second);
else
found->second -= 1;
}
ProcessList::QueryAmount ProcessList::getQueryKindAmount(const IAST::QueryKind & query_kind) const
{
2022-01-10 12:49:53 +00:00
auto found = query_kind_amounts.find(query_kind);
if (found == query_kind_amounts.end())
return 0;
2022-01-10 12:49:53 +00:00
return found->second;
}
}