diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index e304f33efbc..7e8d783836a 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes { extern const int READONLY; extern const int LOGICAL_ERROR; + extern const int CANNOT_KILL; } @@ -138,13 +139,17 @@ public: auto code = process_list.sendCancelToQuery(curr_process.query_id, curr_process.user, true); - if (code != CancellationCode::QueryIsNotInitializedYet && code != CancellationCode::CancelSent) + /// Raise exception if this query is immortal, user have to know + /// This could happen only if query generate streams that don't implement IProfilingBlockInputStream + if (code == CancellationCode::CancelCannotBeSent) + throw Exception("Can't kill query '" + curr_process.query_id + "' it consits of unkillable stages", ErrorCodes::CANNOT_KILL); + else if (code != CancellationCode::QueryIsNotInitializedYet && code != CancellationCode::CancelSent) { curr_process.processed = true; insertResultRow(curr_process.source_num, code, processes_block, res_sample_block, columns); ++num_processed_queries; } - /// Wait if QueryIsNotInitializedYet or CancelSent + /// Wait if CancelSent } /// KILL QUERY could be killed also @@ -194,6 +199,12 @@ BlockIO InterpreterKillQueryQuery::execute() for (const auto & query_desc : queries_to_stop) { auto code = (query.test) ? CancellationCode::Unknown : process_list.sendCancelToQuery(query_desc.query_id, query_desc.user, true); + + /// Raise exception if this query is immortal, user have to know + /// This could happen only if query generate streams that don't implement IProfilingBlockInputStream + if (code == CancellationCode::CancelCannotBeSent) + throw Exception("Can't kill query '" + query_desc.query_id + "' it consits of unkillable stages", ErrorCodes::CANNOT_KILL); + insertResultRow(query_desc.source_num, code, processes_block, header, res_columns); } diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 3c31841d549..20beac13cbf 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -382,8 +382,9 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr } return CancellationCode::CancelCannotBeSent; } - - return CancellationCode::QueryIsNotInitializedYet; + /// Query is not even started + elem->is_killed.store(true); + return CancellationCode::CancelSent; } diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index 87e43162202..d96209d885f 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -191,6 +191,8 @@ public: /// Get query in/out pointers from BlockIO bool tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const; + + bool isKilled() const { return is_killed; } }; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index d949ae3b932..5c4f082eefd 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int QUERY_IS_TOO_LARGE; extern const int INTO_OUTFILE_NOT_ALLOWED; + extern const int QUERY_WAS_CANCELLED; } @@ -204,9 +205,15 @@ static std::tuple executeQueryImpl( auto interpreter = InterpreterFactory::get(ast, context, stage); res = interpreter->execute(); - /// Delayed initialization of query streams (required for KILL QUERY purposes) if (process_list_entry) - (*process_list_entry)->setQueryStreams(res); + { + /// Query was killed before execution + if ((*process_list_entry)->isKilled()) + throw Exception("Query '" + (*process_list_entry)->getInfo().client_info.current_query_id + "' is killed in pending state", + ErrorCodes::QUERY_WAS_CANCELLED); + else + (*process_list_entry)->setQueryStreams(res); + } /// Hold element of process list till end of query execution. res.process_list_entry = process_list_entry; diff --git a/dbms/src/Parsers/ASTKillQueryQuery.cpp b/dbms/src/Parsers/ASTKillQueryQuery.cpp index a8b351cdb39..0b9e6bcf4bc 100644 --- a/dbms/src/Parsers/ASTKillQueryQuery.cpp +++ b/dbms/src/Parsers/ASTKillQueryQuery.cpp @@ -10,7 +10,7 @@ String ASTKillQueryQuery::getID() const void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY "; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY"; formatOnCluster(settings); settings.ostr << " WHERE " << (settings.hilite ? hilite_none : ""); diff --git a/dbms/tests/queries/0_stateless/00417_kill_query.reference b/dbms/tests/queries/0_stateless/00417_kill_query.reference index 7e89d9674db..844ee1838ca 100644 --- a/dbms/tests/queries/0_stateless/00417_kill_query.reference +++ b/dbms/tests/queries/0_stateless/00417_kill_query.reference @@ -1,2 +1,3 @@ SELECT sleep(1) FROM system.numbers LIMIT 4 SELECT sleep(1) FROM system.numbers LIMIT 5 +0 diff --git a/dbms/tests/queries/0_stateless/00417_kill_query.sh b/dbms/tests/queries/0_stateless/00417_kill_query.sh index 04bce8df014..332d0e4591a 100755 --- a/dbms/tests/queries/0_stateless/00417_kill_query.sh +++ b/dbms/tests/queries/0_stateless/00417_kill_query.sh @@ -19,3 +19,30 @@ $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" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 1 TEST" &>/dev/null + + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.cannot_kill_query" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test.cannot_kill_query (x UInt64) ENGINE = MergeTree ORDER BY x" &> /dev/null +$CLICKHOUSE_CLIENT -q "INSERT INTO test.cannot_kill_query SELECT * FROM numbers(10000000)" &> /dev/null + +query_for_pending="SELECT count() FROM test.cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1" +$CLICKHOUSE_CLIENT -q "$query_for_pending" &>/dev/null & + +$CLICKHOUSE_CLIENT -q "ALTER TABLE test.cannot_kill_query MODIFY COLUMN x UInt64" &>/dev/null & + +query_to_kill="SELECT sum(1) FROM test.cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1" +$CLICKHOUSE_CLIENT -q "$query_to_kill" &>/dev/null & + +sleep 1 # just to be sure that 'KILL ...' will be executed after 'SELECT ... WHERE NOT ignore(sleep(1))' + +$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_to_kill'" &>/dev/null & + +# 'SELECT ... WHERE NOT ignore(sleep(1))' is executing much longer than 3 secs, so this sleep doesn't fail test logic +# but guarantees to eliminate flaps, when SELECT from system.process is executed before KILL is completed +sleep 3 + +$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes where query='$query_to_kill'" + +$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_for_pending'" &>/dev/null & # kill pending query + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.cannot_kill_query" &>/dev/null