ClickHouse/dbms/DataStreams/AggregatingSortedBlockInputStream.h

124 lines
4.1 KiB
C++
Raw Normal View History

2014-05-26 16:11:20 +00:00
#pragma once
2015-09-29 19:19:54 +00:00
#include <common/logger_useful.h>
Fix SimpleAggregateFunction for String longer MAX_SMALL_STRING_SIZE SimpleAggregateFunction do not pass arena to the add_function -> getAddressOfAddFunction(), hence next crash happens: (gdb) bt #0 DB::Arena::alloc (size=64, this=0x0) at ../dbms/src/Common/Arena.h:124 #1 DB::SingleValueDataString::changeImpl (this=0x7f97424a27d8, value=..., arena=0x0) at ../dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h:274 #2 0x0000000005ea5319 in DB::AggregateFunctionNullUnary<true>::add (arena=<optimized out>, row_num=<optimized out>, columns=<optimized out>, place=<optimized out>, this=<optimized out>) at ../dbms/src/AggregateFunctions/AggregateFunctionNull.h:43 #3 DB::IAggregateFunctionHelper<DB::AggregateFunctionNullUnary<true> >::addFree (that=<optimized out>, place=<optimized out>, columns=<optimized out>, row_num=<optimized out>, arena=<optimized out>) at ../dbms/src/AggregateFunctions/IAggregateFunction.h:131 #4 0x000000000679772f in DB::AggregatingSortedBlockInputStream::addRow (this=this@entry=0x7f982de19c00, cursor=...) at ../dbms/src/Common/AlignedBuffer.h:31 #5 0x0000000006797faa in DB::AggregatingSortedBlockInputStream::merge (this=this@entry=0x7f982de19c00, merged_columns=..., queue=...) at ../dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp:140 #6 0x0000000006798979 in DB::AggregatingSortedBlockInputStream::readImpl (this=0x7f982de19c00) at ../dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp:78 #7 0x000000000622db55 in DB::IBlockInputStream::read (this=0x7f982de19c00) at ../dbms/src/DataStreams/IBlockInputStream.cpp:56 #8 0x0000000006613bee in DB::MergeTreeDataMergerMutator::mergePartsToTemporaryPart (this=this@entry=0x7f97ec65e1a0, future_part=..., merge_entry=..., time_of_merge=<optimized out>, disk_reservation=<optimized out>, deduplicate=<optimized out>) at /usr/include/c++/8/bits/shared_ptr_base.h:1018 #9 0x000000000658f7a4 in DB::StorageReplicatedMergeTree::tryExecuteMerge (this=0x7f97ec65b810, entry=...) at /usr/include/c++/8/bits/unique_ptr.h:342 #10 0x00000000065940ab in DB::StorageReplicatedMergeTree::executeLogEntry (this=0x7f97ec65b810, entry=...) at ../dbms/src/Storages/StorageReplicatedMergeTree.cpp:910 <snip> (gdb) f 1 (gdb) p MAX_SMALL_STRING_SIZE $1 = 48 (gdb) p capacity $2 = 64 (gdb) p value $3 = {data = 0x7f97242fcbd0 "HHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHH", size = 61} v2: avoid leaking of allocated by Arena memory on the intermediate step Fixes: 8f8d2c048e ("Merge pull request #4629 from bgranvea/simple_aggregate_function")
2019-05-16 17:41:09 +00:00
#include <memory>
2014-05-26 16:11:20 +00:00
#include <Core/ColumnNumbers.h>
#include <DataStreams/MergingSortedBlockInputStream.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Common/AlignedBuffer.h>
2014-05-26 16:11:20 +00:00
namespace DB
{
Fix SimpleAggregateFunction for String longer MAX_SMALL_STRING_SIZE SimpleAggregateFunction do not pass arena to the add_function -> getAddressOfAddFunction(), hence next crash happens: (gdb) bt #0 DB::Arena::alloc (size=64, this=0x0) at ../dbms/src/Common/Arena.h:124 #1 DB::SingleValueDataString::changeImpl (this=0x7f97424a27d8, value=..., arena=0x0) at ../dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h:274 #2 0x0000000005ea5319 in DB::AggregateFunctionNullUnary<true>::add (arena=<optimized out>, row_num=<optimized out>, columns=<optimized out>, place=<optimized out>, this=<optimized out>) at ../dbms/src/AggregateFunctions/AggregateFunctionNull.h:43 #3 DB::IAggregateFunctionHelper<DB::AggregateFunctionNullUnary<true> >::addFree (that=<optimized out>, place=<optimized out>, columns=<optimized out>, row_num=<optimized out>, arena=<optimized out>) at ../dbms/src/AggregateFunctions/IAggregateFunction.h:131 #4 0x000000000679772f in DB::AggregatingSortedBlockInputStream::addRow (this=this@entry=0x7f982de19c00, cursor=...) at ../dbms/src/Common/AlignedBuffer.h:31 #5 0x0000000006797faa in DB::AggregatingSortedBlockInputStream::merge (this=this@entry=0x7f982de19c00, merged_columns=..., queue=...) at ../dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp:140 #6 0x0000000006798979 in DB::AggregatingSortedBlockInputStream::readImpl (this=0x7f982de19c00) at ../dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp:78 #7 0x000000000622db55 in DB::IBlockInputStream::read (this=0x7f982de19c00) at ../dbms/src/DataStreams/IBlockInputStream.cpp:56 #8 0x0000000006613bee in DB::MergeTreeDataMergerMutator::mergePartsToTemporaryPart (this=this@entry=0x7f97ec65e1a0, future_part=..., merge_entry=..., time_of_merge=<optimized out>, disk_reservation=<optimized out>, deduplicate=<optimized out>) at /usr/include/c++/8/bits/shared_ptr_base.h:1018 #9 0x000000000658f7a4 in DB::StorageReplicatedMergeTree::tryExecuteMerge (this=0x7f97ec65b810, entry=...) at /usr/include/c++/8/bits/unique_ptr.h:342 #10 0x00000000065940ab in DB::StorageReplicatedMergeTree::executeLogEntry (this=0x7f97ec65b810, entry=...) at ../dbms/src/Storages/StorageReplicatedMergeTree.cpp:910 <snip> (gdb) f 1 (gdb) p MAX_SMALL_STRING_SIZE $1 = 48 (gdb) p capacity $2 = 64 (gdb) p value $3 = {data = 0x7f97242fcbd0 "HHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHH", size = 61} v2: avoid leaking of allocated by Arena memory on the intermediate step Fixes: 8f8d2c048e ("Merge pull request #4629 from bgranvea/simple_aggregate_function")
2019-05-16 17:41:09 +00:00
class Arena;
2017-05-13 22:19:04 +00:00
/** Merges several sorted streams to one.
* During this for each group of consecutive identical values of the primary key (the columns by which the data is sorted),
* merges them into one row. When merging, the data is pre-aggregated - merge of states of aggregate functions,
* corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type,
2017-08-12 21:02:45 +00:00
* when merged, the first value is selected.
2014-05-26 16:11:20 +00:00
*/
class AggregatingSortedBlockInputStream : public MergingSortedBlockInputStream
{
public:
AggregatingSortedBlockInputStream(
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_);
2014-05-26 16:11:20 +00:00
String getName() const override { return "AggregatingSorted"; }
2014-05-26 16:11:20 +00:00
bool isSortedOutput() const override { return true; }
Block getHeader() const override { return result_header; }
2014-05-26 16:11:20 +00:00
protected:
2017-05-13 22:19:04 +00:00
/// Can return 1 more records than max_block_size.
Block readImpl() override;
2014-05-26 16:11:20 +00:00
private:
Logger * log = &Logger::get("AggregatingSortedBlockInputStream");
2015-01-18 08:25:56 +00:00
2017-05-13 22:19:04 +00:00
/// Read finished.
bool finished = false;
2014-05-26 16:11:20 +00:00
struct SimpleAggregateDescription;
2017-05-13 22:19:04 +00:00
/// Columns with which numbers should be aggregated.
ColumnNumbers column_numbers_to_aggregate;
ColumnNumbers column_numbers_not_to_aggregate;
std::vector<ColumnAggregateFunction *> columns_to_aggregate;
std::vector<SimpleAggregateDescription> columns_to_simple_aggregate;
2014-05-26 16:11:20 +00:00
2019-04-19 13:38:25 +00:00
SharedBlockRowRef current_key; /// The current primary key.
SharedBlockRowRef next_key; /// The primary key of the next row.
2014-05-26 16:11:20 +00:00
Block result_header;
ColumnNumbers converted_lc_columns;
2017-05-13 22:19:04 +00:00
/** We support two different cursors - with Collation and without.
* Templates are used instead of polymorphic SortCursor and calls to virtual functions.
*/
2019-12-22 00:19:07 +00:00
void merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue);
2014-05-26 16:11:20 +00:00
2017-05-13 22:19:04 +00:00
/** Extract all states of aggregate functions and merge them with the current group.
*/
void addRow(SortCursor & cursor);
/** Insert all values of current row for simple aggregate functions
*/
void insertSimpleAggregationResult(MutableColumns & merged_columns);
/// Does SimpleAggregateFunction allocates memory in arena?
bool allocatesMemoryInArena = false;
/// Memory pool for SimpleAggregateFunction
/// (only when allocatesMemoryInArena == true).
Fix SimpleAggregateFunction for String longer MAX_SMALL_STRING_SIZE SimpleAggregateFunction do not pass arena to the add_function -> getAddressOfAddFunction(), hence next crash happens: (gdb) bt #0 DB::Arena::alloc (size=64, this=0x0) at ../dbms/src/Common/Arena.h:124 #1 DB::SingleValueDataString::changeImpl (this=0x7f97424a27d8, value=..., arena=0x0) at ../dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h:274 #2 0x0000000005ea5319 in DB::AggregateFunctionNullUnary<true>::add (arena=<optimized out>, row_num=<optimized out>, columns=<optimized out>, place=<optimized out>, this=<optimized out>) at ../dbms/src/AggregateFunctions/AggregateFunctionNull.h:43 #3 DB::IAggregateFunctionHelper<DB::AggregateFunctionNullUnary<true> >::addFree (that=<optimized out>, place=<optimized out>, columns=<optimized out>, row_num=<optimized out>, arena=<optimized out>) at ../dbms/src/AggregateFunctions/IAggregateFunction.h:131 #4 0x000000000679772f in DB::AggregatingSortedBlockInputStream::addRow (this=this@entry=0x7f982de19c00, cursor=...) at ../dbms/src/Common/AlignedBuffer.h:31 #5 0x0000000006797faa in DB::AggregatingSortedBlockInputStream::merge (this=this@entry=0x7f982de19c00, merged_columns=..., queue=...) at ../dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp:140 #6 0x0000000006798979 in DB::AggregatingSortedBlockInputStream::readImpl (this=0x7f982de19c00) at ../dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp:78 #7 0x000000000622db55 in DB::IBlockInputStream::read (this=0x7f982de19c00) at ../dbms/src/DataStreams/IBlockInputStream.cpp:56 #8 0x0000000006613bee in DB::MergeTreeDataMergerMutator::mergePartsToTemporaryPart (this=this@entry=0x7f97ec65e1a0, future_part=..., merge_entry=..., time_of_merge=<optimized out>, disk_reservation=<optimized out>, deduplicate=<optimized out>) at /usr/include/c++/8/bits/shared_ptr_base.h:1018 #9 0x000000000658f7a4 in DB::StorageReplicatedMergeTree::tryExecuteMerge (this=0x7f97ec65b810, entry=...) at /usr/include/c++/8/bits/unique_ptr.h:342 #10 0x00000000065940ab in DB::StorageReplicatedMergeTree::executeLogEntry (this=0x7f97ec65b810, entry=...) at ../dbms/src/Storages/StorageReplicatedMergeTree.cpp:910 <snip> (gdb) f 1 (gdb) p MAX_SMALL_STRING_SIZE $1 = 48 (gdb) p capacity $2 = 64 (gdb) p value $3 = {data = 0x7f97242fcbd0 "HHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHHH", size = 61} v2: avoid leaking of allocated by Arena memory on the intermediate step Fixes: 8f8d2c048e ("Merge pull request #4629 from bgranvea/simple_aggregate_function")
2019-05-16 17:41:09 +00:00
std::unique_ptr<Arena> arena;
/// Stores information for aggregation of SimpleAggregateFunction columns
struct SimpleAggregateDescription
{
/// An aggregate function 'anyLast', 'sum'...
AggregateFunctionPtr function;
IAggregateFunction::AddFunc add_function;
size_t column_number;
AlignedBuffer state;
bool created = false;
SimpleAggregateDescription(const AggregateFunctionPtr & function_, const size_t column_number_) : function(function_), column_number(column_number_)
{
2019-11-11 08:36:19 +00:00
add_function = function->getAddressOfAddFunction();
state.reset(function->sizeOfData(), function->alignOfData());
}
void createState()
{
if (created)
return;
function->create(state.data());
created = true;
}
void destroyState()
{
if (!created)
return;
function->destroy(state.data());
created = false;
}
/// Explicitly destroy aggregation state if the stream is terminated
~SimpleAggregateDescription()
{
destroyState();
}
SimpleAggregateDescription() = default;
SimpleAggregateDescription(SimpleAggregateDescription &&) = default;
SimpleAggregateDescription(const SimpleAggregateDescription &) = delete;
};
2014-05-26 16:11:20 +00:00
};
}