Improve KILL QUERY. [#CLICKHOUSE-2120]

This commit is contained in:
Vitaliy Lyudvichenko 2017-01-24 16:39:39 +03:00 committed by alexey-milovidov
parent b1bccb0bb2
commit 6bd306ca8c
10 changed files with 74 additions and 42 deletions

View File

@ -18,7 +18,6 @@ public:
private:
String getSelectFromSystemProcessesQuery();
Block getSelectFromSystemProcessesResult();
ASTPtr query_ptr;

View File

@ -1,9 +1,10 @@
#include <DB/Parsers/IAST.h>
#include <DB/Parsers/ASTQueryWithOutput.h>
namespace DB
{
class ASTKillQueryQuery : public IAST
class ASTKillQueryQuery : public ASTQueryWithOutput
{
public:
ASTPtr where_expression;
@ -11,24 +12,13 @@ public:
ASTKillQueryQuery() = default;
ASTKillQueryQuery(const StringRange range_) : IAST(range_) {}
ASTKillQueryQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
ASTPtr clone() const override { return std::make_shared<ASTKillQueryQuery>(*this); }
String getID() const override
{
return "KillQueryQuery_" + (where_expression ? where_expression->getID() : "") + "_" + String(sync ? "SYNC" : "ASYNC");
}
String getID() const override;
void formatImpl(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");
}
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -1,5 +1,6 @@
#include <DB/Interpreters/InterpreterKillQueryQuery.h>
#include <DB/Parsers/ASTKillQueryQuery.h>
#include <DB/Parsers/queryToString.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/Columns/ColumnString.h>
@ -175,12 +176,14 @@ BlockIO InterpreterKillQueryQuery::execute()
{
ASTKillQueryQuery & query = typeid_cast<ASTKillQueryQuery &>(*query_ptr);
BlockIO res_io;
Block processes_block = getSelectFromSystemProcessesResult();
if (!processes_block)
return res_io;
ProcessList & process_list = context.getProcessList();
QueryDescriptors queries_to_stop = extractQueriesExceptMeAndCheckAccess(processes_block, context);
BlockIO res_io;
res_io.in_sample = processes_block.cloneEmpty();
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;
}
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()
{
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";
BlockIO system_processes_io = executeQuery(system_processes_query, context, true);
Block 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);
if (res && system_processes_io.in->read())
throw Exception("Expected one block from input stream", ErrorCodes::LOGICAL_ERROR);
return res;
}

View File

@ -191,12 +191,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
auto interpreter = InterpreterFactory::get(ast, context, stage);
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)
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)
{

View File

@ -3,4 +3,19 @@
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");
}
}

View File

@ -19,7 +19,7 @@ namespace DB
bool ParserKillQueryQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
auto query = std::make_shared<ASTKillQueryQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTKillQueryQuery>();
ParserWhiteSpaceOrComments ws;
@ -54,6 +54,10 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
else
expected = "[SYNC|ASYNC]";
ws.ignore(pos, end);
query->range = StringRange(begin, pos);
node = std::move(query);
return true;

View File

@ -8,7 +8,6 @@
#include <DB/Parsers/ParserUseQuery.h>
#include <DB/Parsers/ParserSetQuery.h>
#include <DB/Parsers/ParserAlterQuery.h>
#include <DB/Parsers/ParserKillQueryQuery.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;
ParserSetQuery set_p;
ParserOptimizeQuery optimize_p;
ParserKillQueryQuery kill_query_p;
// ParserMultiQuery multi_p;
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)
|| use_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)
|| kill_query_p.parse(pos, end, node, max_parsed_pos, expected);
|| optimize_p.parse(pos, end, node, max_parsed_pos, expected);
/* || multi_p.parse(pos, end, node, max_parsed_pos, 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;
}

View File

@ -4,6 +4,7 @@
#include <DB/Parsers/ParserTablePropertiesQuery.h>
#include <DB/Parsers/ParserShowProcesslistQuery.h>
#include <DB/Parsers/ParserCheckQuery.h>
#include <DB/Parsers/ParserKillQueryQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ExpressionElementParsers.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;
ParserShowProcesslistQuery show_processlist_p;
ParserCheckQuery check_p;
ParserKillQueryQuery kill_query_p;
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)
|| table_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)
return false;

View File

@ -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

View 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"