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
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/ColumnNumbers.h>
|
|
|
|
#include <DataStreams/MergingSortedBlockInputStream.h>
|
|
|
|
#include <AggregateFunctions/IAggregateFunction.h>
|
|
|
|
#include <Columns/ColumnAggregateFunction.h>
|
2019-03-08 16:49:10 +00:00
|
|
|
#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:
|
2018-04-07 01:46:50 +00:00
|
|
|
AggregatingSortedBlockInputStream(
|
2019-02-19 01:47:27 +00:00
|
|
|
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_);
|
2014-05-26 16:11:20 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return "AggregatingSorted"; }
|
2014-05-26 16:11:20 +00:00
|
|
|
|
2017-04-07 19:47:21 +00:00
|
|
|
bool isSortedOutput() const override { return true; }
|
|
|
|
|
2020-01-10 14:01:24 +00:00
|
|
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
Block readImpl() override;
|
2014-05-26 16:11:20 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
Logger * log = &Logger::get("AggregatingSortedBlockInputStream");
|
2015-01-18 08:25:56 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Read finished.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool finished = false;
|
2014-05-26 16:11:20 +00:00
|
|
|
|
2019-03-08 16:49:10 +00:00
|
|
|
struct SimpleAggregateDescription;
|
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Columns with which numbers should be aggregated.
|
2017-04-01 07:20:54 +00:00
|
|
|
ColumnNumbers column_numbers_to_aggregate;
|
|
|
|
ColumnNumbers column_numbers_not_to_aggregate;
|
|
|
|
std::vector<ColumnAggregateFunction *> columns_to_aggregate;
|
2019-03-08 16:49:10 +00:00
|
|
|
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
|
|
|
|
2020-01-10 14:01:24 +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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2017-12-15 00:06:56 +00:00
|
|
|
void addRow(SortCursor & cursor);
|
2019-03-08 16:49:10 +00:00
|
|
|
|
|
|
|
/** Insert all values of current row for simple aggregate functions
|
|
|
|
*/
|
|
|
|
void insertSimpleAggregationResult(MutableColumns & merged_columns);
|
|
|
|
|
2019-05-18 14:29:33 +00:00
|
|
|
/// 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;
|
|
|
|
|
2019-03-08 16:49:10 +00:00
|
|
|
/// 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();
|
2019-03-08 16:49:10 +00:00
|
|
|
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
|
|
|
};
|
|
|
|
|
|
|
|
}
|