mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 13:32:13 +00:00
Remove unnecessary flag
This commit is contained in:
parent
1375733643
commit
2c055480d6
@ -42,7 +42,7 @@ void RemoteSource::setStorageLimits(const std::shared_ptr<const StorageLimitsLis
|
||||
ISource::Status RemoteSource::prepare()
|
||||
{
|
||||
/// Check if query was cancelled before returning Async status. Otherwise it may lead to infinite loop.
|
||||
if (was_query_canceled)
|
||||
if (isCancelled())
|
||||
{
|
||||
getPort().finish();
|
||||
return Status::Finished;
|
||||
@ -67,7 +67,7 @@ ISource::Status RemoteSource::prepare()
|
||||
std::optional<Chunk> RemoteSource::tryGenerate()
|
||||
{
|
||||
/// onCancel() will do the cancel if the query was sent.
|
||||
if (was_query_canceled)
|
||||
if (isCancelled())
|
||||
return {};
|
||||
|
||||
if (!was_query_sent)
|
||||
@ -169,7 +169,6 @@ std::optional<Chunk> RemoteSource::tryGenerate()
|
||||
|
||||
void RemoteSource::onCancel()
|
||||
{
|
||||
was_query_canceled = true;
|
||||
query_executor->cancel();
|
||||
}
|
||||
|
||||
@ -177,7 +176,6 @@ void RemoteSource::onUpdatePorts()
|
||||
{
|
||||
if (getPort().isFinished())
|
||||
{
|
||||
was_query_canceled = true;
|
||||
query_executor->finish();
|
||||
}
|
||||
}
|
||||
|
@ -39,7 +39,6 @@ protected:
|
||||
void onCancel() override;
|
||||
|
||||
private:
|
||||
std::atomic<bool> was_query_canceled = false;
|
||||
bool was_query_sent = false;
|
||||
bool add_aggregation_info = false;
|
||||
RemoteQueryExecutorPtr query_executor;
|
||||
|
Loading…
Reference in New Issue
Block a user