mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #417 from yandex/revert-384-query-kill
Revert "KILL QUERY"
This commit is contained in:
commit
94c54cf629
@ -84,9 +84,8 @@ protected:
|
||||
BlockInputStreamPtr query_stream_in;
|
||||
BlockOutputStreamPtr query_stream_out;
|
||||
|
||||
/// Abovemetioned streams have delayed initialization, this flag indicates thier initialization
|
||||
/// It is better to use atomic (instead of raw bool) with tryGet/setQueryStreams() thread-safe methods despite that
|
||||
/// now in all contexts ProcessListElement is always used under ProcessList::mutex (and raw bool is also Ok)
|
||||
/// Abovemetioned streams haved delayed initialization, use this flag to track initialization
|
||||
/// Can be set to true by single thread, can be read from multiple
|
||||
std::atomic<bool> query_streams_initialized{false};
|
||||
|
||||
public:
|
||||
@ -147,7 +146,7 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
/// Copies pointers to in/out streams, it can be called once
|
||||
/// Copies pointers to in/out streams
|
||||
void setQueryStreams(const BlockIO & io);
|
||||
/// Get query in/out pointers
|
||||
bool tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const;
|
||||
@ -265,13 +264,13 @@ public:
|
||||
{
|
||||
NotFound = 0, /// already cancelled
|
||||
QueryIsNotInitializedYet = 1,
|
||||
CancelCannotBeSent = 2,
|
||||
CancelSent = 3,
|
||||
CancelCannotBeSended = 2,
|
||||
CancelSended = 3,
|
||||
Unknown
|
||||
};
|
||||
|
||||
/// Try call cancel() for input and output streams of query with specified id and user
|
||||
CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user);
|
||||
CancellationCode sendCancelToQuery(const String & query_id, const String & intial_user);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -7,9 +7,9 @@ namespace DB
|
||||
class ASTKillQueryQuery : public ASTQueryWithOutput
|
||||
{
|
||||
public:
|
||||
ASTPtr where_expression; // expression to filter processes from system.processes table
|
||||
bool sync = false; // SYNC or ASYNC mode
|
||||
bool test = false; // does it TEST mode? (doesn't cancel queries just checks and shows them)
|
||||
ASTPtr where_expression;
|
||||
bool sync = false;
|
||||
bool test = false;
|
||||
|
||||
ASTKillQueryQuery() = default;
|
||||
|
||||
|
@ -32,9 +32,9 @@ static const char * cancellationCodeToStatus(CancellationCode code)
|
||||
return "finished";
|
||||
case CancellationCode::QueryIsNotInitializedYet:
|
||||
return "pending";
|
||||
case CancellationCode::CancelCannotBeSent:
|
||||
case CancellationCode::CancelCannotBeSended:
|
||||
return "error";
|
||||
case CancellationCode::CancelSent:
|
||||
case CancellationCode::CancelSended:
|
||||
return "waiting";
|
||||
default:
|
||||
return "unknown_status";
|
||||
@ -141,13 +141,13 @@ public:
|
||||
|
||||
auto code = process_list.sendCancelToQuery(curr_process.query_id, curr_process.user);
|
||||
|
||||
if (code != CancellationCode::QueryIsNotInitializedYet && code != CancellationCode::CancelSent)
|
||||
if (code != CancellationCode::QueryIsNotInitializedYet && code != CancellationCode::CancelSended)
|
||||
{
|
||||
curr_process.processed = true;
|
||||
insertResultRow(curr_process.source_num, code, processes_block, res);
|
||||
++num_processed_queries;
|
||||
}
|
||||
/// Wait if QueryIsNotInitializedYet or CancelSent
|
||||
/// Wait if QueryIsNotInitializedYet or CancelSended
|
||||
}
|
||||
|
||||
/// KILL QUERY could be killed also
|
||||
|
@ -20,7 +20,7 @@ ProcessList::EntryPtr ProcessList::insert(
|
||||
const String & query_, const IAST * ast, const ClientInfo & client_info, const Settings & settings)
|
||||
{
|
||||
EntryPtr res;
|
||||
bool is_kill_query = ast && typeid_cast<const ASTKillQueryQuery *>(ast);
|
||||
bool is_kill_query = ast && dynamic_cast<const ASTKillQueryQuery *>(ast);
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
@ -151,7 +151,7 @@ void ProcessListElement::setQueryStreams(const BlockIO & io)
|
||||
{
|
||||
query_stream_in = io.in;
|
||||
query_stream_out = io.out;
|
||||
query_streams_initialized = true; // forces strict memory ordering
|
||||
query_streams_initialized = true;
|
||||
}
|
||||
|
||||
bool ProcessListElement::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const
|
||||
@ -229,9 +229,9 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr
|
||||
if (input_stream && (input_stream_casted = dynamic_cast<IProfilingBlockInputStream *>(input_stream.get())))
|
||||
{
|
||||
input_stream_casted->cancel();
|
||||
return CancellationCode::CancelSent;
|
||||
return CancellationCode::CancelSended;
|
||||
}
|
||||
return CancellationCode::CancelCannotBeSent;
|
||||
return CancellationCode::CancelCannotBeSended;
|
||||
}
|
||||
return CancellationCode::QueryIsNotInitializedYet;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user