mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-18 13:42:02 +00:00
AggregatingTransform initGenerate race condition fix
This commit is contained in:
parent
21e0cb0b18
commit
0859fc8de9
@ -623,7 +623,9 @@ IProcessor::Status AggregatingTransform::prepare()
|
||||
void AggregatingTransform::work()
|
||||
{
|
||||
if (is_consume_finished)
|
||||
{
|
||||
initGenerate();
|
||||
}
|
||||
else
|
||||
{
|
||||
consume(std::move(current_chunk));
|
||||
@ -676,11 +678,14 @@ void AggregatingTransform::consume(Chunk chunk)
|
||||
|
||||
void AggregatingTransform::initGenerate()
|
||||
{
|
||||
if (is_generate_initialized)
|
||||
if (is_generate_initialized.load(std::memory_order_acquire))
|
||||
return;
|
||||
|
||||
std::lock_guard lock(snapshot_mutex);
|
||||
is_generate_initialized = true;
|
||||
if (is_generate_initialized.load(std::memory_order_relaxed))
|
||||
return;
|
||||
|
||||
is_generate_initialized.store(true, std::memory_order_release);
|
||||
|
||||
/// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation.
|
||||
/// To do this, we pass a block with zero rows to aggregate.
|
||||
|
@ -217,7 +217,7 @@ private:
|
||||
UInt64 src_rows = 0;
|
||||
UInt64 src_bytes = 0;
|
||||
|
||||
bool is_generate_initialized = false;
|
||||
std::atomic<bool> is_generate_initialized = false;
|
||||
bool is_consume_finished = false;
|
||||
bool is_pipeline_created = false;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user