mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
Do not generate empty chunks in AggregatingInOrderTransform
This is just a micro optimization and it should not affect anything, real fixes are in separate patches (previous and next). Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
parent
d9a64d1f86
commit
06402386eb
@ -255,6 +255,8 @@ void AggregatingInOrderTransform::generate()
|
|||||||
res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]);
|
res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]);
|
||||||
|
|
||||||
to_push_chunk = convertToChunk(res);
|
to_push_chunk = convertToChunk(res);
|
||||||
|
if (!to_push_chunk.getNumRows())
|
||||||
|
return;
|
||||||
|
|
||||||
/// Clear arenas to allow to free them, when chunk will reach the end of pipeline.
|
/// Clear arenas to allow to free them, when chunk will reach the end of pipeline.
|
||||||
/// It's safe clear them here, because columns with aggregate functions already holds them.
|
/// It's safe clear them here, because columns with aggregate functions already holds them.
|
||||||
|
Loading…
Reference in New Issue
Block a user