Add requested changes. [#CLICKHOUSE-2120]

This commit is contained in:
Vitaliy Lyudvichenko 2017-01-25 18:31:25 +03:00 committed by alexey-milovidov
parent 891437e531
commit 55390715b1
4 changed files with 18 additions and 17 deletions

View File

@ -84,8 +84,9 @@ protected:
BlockInputStreamPtr query_stream_in;
BlockOutputStreamPtr query_stream_out;
/// Abovemetioned streams haved delayed initialization, use this flag to track initialization
/// Can be set to true by single thread, can be read from multiple
/// 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)
std::atomic<bool> query_streams_initialized{false};
public:
@ -146,7 +147,7 @@ public:
return res;
}
/// Copies pointers to in/out streams
/// Copies pointers to in/out streams, it can be called once
void setQueryStreams(const BlockIO & io);
/// Get query in/out pointers
bool tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const;
@ -264,13 +265,13 @@ public:
{
NotFound = 0, /// already cancelled
QueryIsNotInitializedYet = 1,
CancelCannotBeSended = 2,
CancelSended = 3,
CancelCannotBeSent = 2,
CancelSent = 3,
Unknown
};
/// Try call cancel() for input and output streams of query with specified id and user
CancellationCode sendCancelToQuery(const String & query_id, const String & intial_user);
CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user);
};
}

View File

@ -7,9 +7,9 @@ namespace DB
class ASTKillQueryQuery : public ASTQueryWithOutput
{
public:
ASTPtr where_expression;
bool sync = false;
bool test = false;
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)
ASTKillQueryQuery() = default;

View File

@ -32,9 +32,9 @@ static const char * cancellationCodeToStatus(CancellationCode code)
return "finished";
case CancellationCode::QueryIsNotInitializedYet:
return "pending";
case CancellationCode::CancelCannotBeSended:
case CancellationCode::CancelCannotBeSent:
return "error";
case CancellationCode::CancelSended:
case CancellationCode::CancelSent:
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::CancelSended)
if (code != CancellationCode::QueryIsNotInitializedYet && code != CancellationCode::CancelSent)
{
curr_process.processed = true;
insertResultRow(curr_process.source_num, code, processes_block, res);
++num_processed_queries;
}
/// Wait if QueryIsNotInitializedYet or CancelSended
/// Wait if QueryIsNotInitializedYet or CancelSent
}
/// KILL QUERY could be killed also

View File

@ -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 && dynamic_cast<const ASTKillQueryQuery *>(ast);
bool is_kill_query = ast && typeid_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;
query_streams_initialized = true; // forces strict memory ordering
}
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::CancelSended;
return CancellationCode::CancelSent;
}
return CancellationCode::CancelCannotBeSended;
return CancellationCode::CancelCannotBeSent;
}
return CancellationCode::QueryIsNotInitializedYet;
}