mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Improve KILL QUERY. [#CLICKHOUSE-2120]
This commit is contained in:
parent
b1bccb0bb2
commit
6bd306ca8c
@ -18,7 +18,6 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
|
|
||||||
String getSelectFromSystemProcessesQuery();
|
|
||||||
Block getSelectFromSystemProcessesResult();
|
Block getSelectFromSystemProcessesResult();
|
||||||
|
|
||||||
ASTPtr query_ptr;
|
ASTPtr query_ptr;
|
||||||
|
@ -1,9 +1,10 @@
|
|||||||
#include <DB/Parsers/IAST.h>
|
#include <DB/Parsers/IAST.h>
|
||||||
|
#include <DB/Parsers/ASTQueryWithOutput.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class ASTKillQueryQuery : public IAST
|
class ASTKillQueryQuery : public ASTQueryWithOutput
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ASTPtr where_expression;
|
ASTPtr where_expression;
|
||||||
@ -11,24 +12,13 @@ public:
|
|||||||
|
|
||||||
ASTKillQueryQuery() = default;
|
ASTKillQueryQuery() = default;
|
||||||
|
|
||||||
ASTKillQueryQuery(const StringRange range_) : IAST(range_) {}
|
ASTKillQueryQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
|
||||||
|
|
||||||
ASTPtr clone() const override { return std::make_shared<ASTKillQueryQuery>(*this); }
|
ASTPtr clone() const override { return std::make_shared<ASTKillQueryQuery>(*this); }
|
||||||
|
|
||||||
String getID() const override
|
String getID() const override;
|
||||||
{
|
|
||||||
return "KillQueryQuery_" + (where_expression ? where_expression->getID() : "") + "_" + String(sync ? "SYNC" : "ASYNC");
|
|
||||||
}
|
|
||||||
|
|
||||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||||
{
|
|
||||||
settings.ostr << "KILL QUERY WHERE ";
|
|
||||||
|
|
||||||
if (where_expression)
|
|
||||||
where_expression->formatImpl(settings, state, frame);
|
|
||||||
|
|
||||||
settings.ostr << (sync ? " SYNC" : " ASYNC");
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <DB/Interpreters/InterpreterKillQueryQuery.h>
|
#include <DB/Interpreters/InterpreterKillQueryQuery.h>
|
||||||
#include <DB/Parsers/ASTKillQueryQuery.h>
|
#include <DB/Parsers/ASTKillQueryQuery.h>
|
||||||
|
#include <DB/Parsers/queryToString.h>
|
||||||
#include <DB/Interpreters/ProcessList.h>
|
#include <DB/Interpreters/ProcessList.h>
|
||||||
#include <DB/Interpreters/executeQuery.h>
|
#include <DB/Interpreters/executeQuery.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
@ -175,12 +176,14 @@ BlockIO InterpreterKillQueryQuery::execute()
|
|||||||
{
|
{
|
||||||
ASTKillQueryQuery & query = typeid_cast<ASTKillQueryQuery &>(*query_ptr);
|
ASTKillQueryQuery & query = typeid_cast<ASTKillQueryQuery &>(*query_ptr);
|
||||||
|
|
||||||
|
BlockIO res_io;
|
||||||
Block processes_block = getSelectFromSystemProcessesResult();
|
Block processes_block = getSelectFromSystemProcessesResult();
|
||||||
|
if (!processes_block)
|
||||||
|
return res_io;
|
||||||
|
|
||||||
ProcessList & process_list = context.getProcessList();
|
ProcessList & process_list = context.getProcessList();
|
||||||
QueryDescriptors queries_to_stop = extractQueriesExceptMeAndCheckAccess(processes_block, context);
|
QueryDescriptors queries_to_stop = extractQueriesExceptMeAndCheckAccess(processes_block, context);
|
||||||
|
|
||||||
BlockIO res_io;
|
|
||||||
res_io.in_sample = processes_block.cloneEmpty();
|
res_io.in_sample = processes_block.cloneEmpty();
|
||||||
res_io.in_sample.insert(0, {std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "kill_status"});
|
res_io.in_sample.insert(0, {std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "kill_status"});
|
||||||
|
|
||||||
@ -205,28 +208,18 @@ BlockIO InterpreterKillQueryQuery::execute()
|
|||||||
return res_io;
|
return res_io;
|
||||||
}
|
}
|
||||||
|
|
||||||
String InterpreterKillQueryQuery::getSelectFromSystemProcessesQuery()
|
|
||||||
{
|
|
||||||
std::stringstream system_processes_query;
|
|
||||||
system_processes_query << "SELECT query_id, user, query FROM system.processes WHERE ";
|
|
||||||
|
|
||||||
IAST::FormatSettings ast_format_settings(system_processes_query, false, true);
|
|
||||||
static_cast<ASTKillQueryQuery &>(*query_ptr).where_expression->format(ast_format_settings);
|
|
||||||
|
|
||||||
return system_processes_query.str();
|
|
||||||
}
|
|
||||||
|
|
||||||
Block InterpreterKillQueryQuery::getSelectFromSystemProcessesResult()
|
Block InterpreterKillQueryQuery::getSelectFromSystemProcessesResult()
|
||||||
{
|
{
|
||||||
String system_processes_query = getSelectFromSystemProcessesQuery();
|
String system_processes_query = "SELECT query_id, user, query FROM system.processes WHERE "
|
||||||
|
+ queryToString(static_cast<ASTKillQueryQuery &>(*query_ptr).where_expression);
|
||||||
|
|
||||||
// std::cerr << "executing: " << system_processes_query << "\n";
|
// std::cerr << "executing: " << system_processes_query << "\n";
|
||||||
|
|
||||||
BlockIO system_processes_io = executeQuery(system_processes_query, context, true);
|
BlockIO system_processes_io = executeQuery(system_processes_query, context, true);
|
||||||
Block res = system_processes_io.in->read();
|
Block res = system_processes_io.in->read();
|
||||||
|
|
||||||
if (!res || system_processes_io.in->read())
|
if (res && system_processes_io.in->read())
|
||||||
throw Exception("Expected only one block from input stream", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Expected one block from input stream", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -191,12 +191,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
auto interpreter = InterpreterFactory::get(ast, context, stage);
|
auto interpreter = InterpreterFactory::get(ast, context, stage);
|
||||||
res = interpreter->execute();
|
res = interpreter->execute();
|
||||||
|
|
||||||
/// Hold element of process list till end of query execution.
|
|
||||||
res.process_list_entry = process_list_entry;
|
|
||||||
|
|
||||||
/// Delayed initialization of query streams (required for KILL QUERY purposes)
|
/// Delayed initialization of query streams (required for KILL QUERY purposes)
|
||||||
if (!internal)
|
if (!internal)
|
||||||
(*res.process_list_entry)->setQueryStreams(res);
|
(*process_list_entry)->setQueryStreams(res);
|
||||||
|
|
||||||
|
/// Hold element of process list till end of query execution.
|
||||||
|
res.process_list_entry = process_list_entry;
|
||||||
|
|
||||||
if (res.in)
|
if (res.in)
|
||||||
{
|
{
|
||||||
|
@ -3,4 +3,19 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
String ASTKillQueryQuery::getID() const
|
||||||
|
{
|
||||||
|
return "KillQueryQuery_" + (where_expression ? where_expression->getID() : "") + "_" + String(sync ? "SYNC" : "ASYNC");
|
||||||
|
}
|
||||||
|
|
||||||
|
void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||||
|
{
|
||||||
|
settings.ostr << "KILL QUERY WHERE ";
|
||||||
|
|
||||||
|
if (where_expression)
|
||||||
|
where_expression->formatImpl(settings, state, frame);
|
||||||
|
|
||||||
|
settings.ostr << (sync ? " SYNC" : " ASYNC");
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -19,7 +19,7 @@ namespace DB
|
|||||||
bool ParserKillQueryQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
bool ParserKillQueryQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||||
{
|
{
|
||||||
Pos begin = pos;
|
Pos begin = pos;
|
||||||
auto query = std::make_shared<ASTKillQueryQuery>(StringRange(begin, pos));
|
auto query = std::make_shared<ASTKillQueryQuery>();
|
||||||
|
|
||||||
ParserWhiteSpaceOrComments ws;
|
ParserWhiteSpaceOrComments ws;
|
||||||
|
|
||||||
@ -54,6 +54,10 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
|
|||||||
else
|
else
|
||||||
expected = "[SYNC|ASYNC]";
|
expected = "[SYNC|ASYNC]";
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
query->range = StringRange(begin, pos);
|
||||||
|
|
||||||
node = std::move(query);
|
node = std::move(query);
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
|
@ -8,7 +8,6 @@
|
|||||||
#include <DB/Parsers/ParserUseQuery.h>
|
#include <DB/Parsers/ParserUseQuery.h>
|
||||||
#include <DB/Parsers/ParserSetQuery.h>
|
#include <DB/Parsers/ParserSetQuery.h>
|
||||||
#include <DB/Parsers/ParserAlterQuery.h>
|
#include <DB/Parsers/ParserAlterQuery.h>
|
||||||
#include <DB/Parsers/ParserKillQueryQuery.h>
|
|
||||||
//#include <DB/Parsers/ParserMultiQuery.h>
|
//#include <DB/Parsers/ParserMultiQuery.h>
|
||||||
|
|
||||||
|
|
||||||
@ -27,7 +26,6 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
|
|||||||
ParserUseQuery use_p;
|
ParserUseQuery use_p;
|
||||||
ParserSetQuery set_p;
|
ParserSetQuery set_p;
|
||||||
ParserOptimizeQuery optimize_p;
|
ParserOptimizeQuery optimize_p;
|
||||||
ParserKillQueryQuery kill_query_p;
|
|
||||||
// ParserMultiQuery multi_p;
|
// ParserMultiQuery multi_p;
|
||||||
|
|
||||||
bool res = query_with_output_p.parse(pos, end, node, max_parsed_pos, expected)
|
bool res = query_with_output_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
@ -38,12 +36,11 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
|
|||||||
|| alter_p.parse(pos, end, node, max_parsed_pos, expected)
|
|| alter_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
|| use_p.parse(pos, end, node, max_parsed_pos, expected)
|
|| use_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
|| set_p.parse(pos, end, node, max_parsed_pos, expected)
|
|| set_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
|| optimize_p.parse(pos, end, node, max_parsed_pos, expected)
|
|| optimize_p.parse(pos, end, node, max_parsed_pos, expected);
|
||||||
|| kill_query_p.parse(pos, end, node, max_parsed_pos, expected);
|
|
||||||
/* || multi_p.parse(pos, end, node, max_parsed_pos, expected)*/;
|
/* || multi_p.parse(pos, end, node, max_parsed_pos, expected)*/;
|
||||||
|
|
||||||
if (!res && (!expected || !*expected))
|
if (!res && (!expected || !*expected))
|
||||||
expected = "One of: SHOW TABLES, SHOW DATABASES, SHOW CREATE TABLE, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE, EXISTS, DESCRIBE, DESC, ALTER, SHOW PROCESSLIST, CHECK, KILL QUERY opening curly brace";
|
expected = "One of: SHOW TABLES, SHOW DATABASES, SHOW CREATE TABLE, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE, EXISTS, DESCRIBE, DESC, ALTER, SHOW PROCESSLIST, CHECK, KILL QUERY, opening curly brace";
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <DB/Parsers/ParserTablePropertiesQuery.h>
|
#include <DB/Parsers/ParserTablePropertiesQuery.h>
|
||||||
#include <DB/Parsers/ParserShowProcesslistQuery.h>
|
#include <DB/Parsers/ParserShowProcesslistQuery.h>
|
||||||
#include <DB/Parsers/ParserCheckQuery.h>
|
#include <DB/Parsers/ParserCheckQuery.h>
|
||||||
|
#include <DB/Parsers/ParserKillQueryQuery.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||||
#include <DB/Common/typeid_cast.h>
|
#include <DB/Common/typeid_cast.h>
|
||||||
@ -19,6 +20,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
|
|||||||
ParserTablePropertiesQuery table_p;
|
ParserTablePropertiesQuery table_p;
|
||||||
ParserShowProcesslistQuery show_processlist_p;
|
ParserShowProcesslistQuery show_processlist_p;
|
||||||
ParserCheckQuery check_p;
|
ParserCheckQuery check_p;
|
||||||
|
ParserKillQueryQuery kill_query_p;
|
||||||
|
|
||||||
ASTPtr query;
|
ASTPtr query;
|
||||||
|
|
||||||
@ -26,7 +28,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
|
|||||||
|| show_tables_p.parse(pos, end, query, max_parsed_pos, expected)
|
|| show_tables_p.parse(pos, end, query, max_parsed_pos, expected)
|
||||||
|| table_p.parse(pos, end, query, max_parsed_pos, expected)
|
|| table_p.parse(pos, end, query, max_parsed_pos, expected)
|
||||||
|| show_processlist_p.parse(pos, end, query, max_parsed_pos, expected)
|
|| show_processlist_p.parse(pos, end, query, max_parsed_pos, expected)
|
||||||
|| check_p.parse(pos, end, query, max_parsed_pos, expected);
|
|| check_p.parse(pos, end, query, max_parsed_pos, expected)
|
||||||
|
|| kill_query_p.parse(pos, end, query, max_parsed_pos, expected);
|
||||||
|
|
||||||
if (!parsed)
|
if (!parsed)
|
||||||
return false;
|
return false;
|
||||||
|
@ -0,0 +1,6 @@
|
|||||||
|
SELECT sleep(1) FROM system.numbers LIMIT 6
|
||||||
|
SELECT sleep(2) FROM system.numbers LIMIT 3
|
||||||
|
SELECT sleep(3) FROM system.numbers LIMIT 2
|
||||||
|
0
|
||||||
|
SELECT sleep(1) FROM system.numbers LIMIT 999
|
||||||
|
0
|
25
dbms/tests/queries/0_stateless/00417_kill_query.sh
Executable file
25
dbms/tests/queries/0_stateless/00417_kill_query.sh
Executable file
@ -0,0 +1,25 @@
|
|||||||
|
#!/bin/bash
|
||||||
|
set -e
|
||||||
|
|
||||||
|
QUERY_FIELND_NUM=4
|
||||||
|
|
||||||
|
# should be quite deterministic
|
||||||
|
clickhouse-client --max_block_size=1 -q "SELECT sleep(3) FROM system.numbers LIMIT 2" &>/dev/null &
|
||||||
|
clickhouse-client --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 6" &>/dev/null &
|
||||||
|
clickhouse-client --max_block_size=1 -q "SELECT sleep(2) FROM system.numbers LIMIT 3" &>/dev/null &
|
||||||
|
clickhouse-client --max_block_size=1 -q "SELECT 'trash', sleep(2) FROM system.numbers LIMIT 3" &>/dev/null &
|
||||||
|
sleep 1 # wait initialization
|
||||||
|
clickhouse-client -q "KILL QUERY WHERE query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELND_NUM
|
||||||
|
clickhouse-client -q "SELECT countIf(query LIKE 'SELECT sleep(%') FROM system.processes"
|
||||||
|
|
||||||
|
|
||||||
|
clickhouse-client --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 999" &>/dev/null &
|
||||||
|
sleep 1
|
||||||
|
clickhouse-client -q "KILL QUERY WHERE query = 'SELECT sleep(1) FROM system.numbers LIMIT 999' ASYNC" | cut -f $QUERY_FIELND_NUM
|
||||||
|
sleep 1 # wait cancelling
|
||||||
|
clickhouse-client -q "SELECT countIf(query = 'SELECT sleep(1) FROM system.numbers LIMIT 999') FROM system.processes"
|
||||||
|
|
||||||
|
|
||||||
|
clickhouse-client -q "KILL QUERY WHERE 0 ASYNC"
|
||||||
|
clickhouse-client -q "KILL QUERY WHERE 0 FORMAT TabSeparated"
|
||||||
|
clickhouse-client -q "KILL QUERY WHERE 0 SYNC FORMAT TabSeparated"
|
Loading…
Reference in New Issue
Block a user