ClickHouse/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp

253 lines
8.1 KiB
C++
Raw Normal View History

#include <DataStreams/AggregatingSortedBlockInputStream.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.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 <Common/Arena.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
#include <DataTypes/DataTypeLowCardinality.h>
2014-05-26 16:11:20 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2014-05-26 16:11:20 +00:00
class RemovingLowCardinalityBlockInputStream : public IBlockInputStream
{
public:
RemovingLowCardinalityBlockInputStream(BlockInputStreamPtr input_, ColumnNumbers positions_)
: input(std::move(input_)), positions(std::move(positions_))
{
header = transform(input->getHeader());
}
Block transform(Block block)
{
if (block)
{
for (auto & pos : positions)
{
auto & col = block.safeGetByPosition(pos);
col.column = recursiveRemoveLowCardinality(col.column);
col.type = recursiveRemoveLowCardinality(col.type);
}
}
return block;
}
String getName() const override { return "RemovingLowCardinality"; }
Block getHeader() const override { return header; }
const BlockMissingValues & getMissingValues() const override { return input->getMissingValues(); }
bool isSortedOutput() const override { return input->isSortedOutput(); }
const SortDescription & getSortDescription() const override { return input->getSortDescription(); }
protected:
Block readImpl() override { return transform(input->read()); }
private:
Block header;
BlockInputStreamPtr input;
ColumnNumbers positions;
};
AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream(
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_)
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_)
{
/// Fill in the column numbers that need to be aggregated.
for (size_t i = 0; i < num_columns; ++i)
{
ColumnWithTypeAndName & column = header.safeGetByPosition(i);
/// We leave only states of aggregate functions.
if (!dynamic_cast<const DataTypeAggregateFunction *>(column.type.get()) && !dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(column.type->getCustomName()))
{
column_numbers_not_to_aggregate.push_back(i);
continue;
}
/// Included into PK?
SortDescription::const_iterator it = description.begin();
for (; it != description.end(); ++it)
if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i))
break;
if (it != description.end())
{
column_numbers_not_to_aggregate.push_back(i);
continue;
}
if (auto simple_aggr = dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(column.type->getCustomName()))
2019-03-11 08:24:52 +00:00
{
// simple aggregate function
SimpleAggregateDescription desc{simple_aggr->getFunction(), i};
if (desc.function->allocatesMemoryInArena())
allocatesMemoryInArena = true;
columns_to_simple_aggregate.emplace_back(std::move(desc));
if (recursiveRemoveLowCardinality(column.type).get() != column.type.get())
converted_lc_columns.emplace_back(i);
2019-03-11 08:24:52 +00:00
}
else
{
// standard aggregate function
column_numbers_to_aggregate.push_back(i);
}
}
result_header = header;
if (!converted_lc_columns.empty())
{
for (auto & input : children)
input = std::make_shared<RemovingLowCardinalityBlockInputStream>(input, converted_lc_columns);
header = children.at(0)->getHeader();
}
}
2014-05-26 16:11:20 +00:00
Block AggregatingSortedBlockInputStream::readImpl()
{
if (finished)
return Block();
MutableColumns merged_columns;
init(merged_columns);
if (has_collation)
throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR);
if (merged_columns.empty())
return Block();
columns_to_aggregate.resize(column_numbers_to_aggregate.size());
for (size_t i = 0, size = columns_to_aggregate.size(); i < size; ++i)
columns_to_aggregate[i] = typeid_cast<ColumnAggregateFunction *>(merged_columns[column_numbers_to_aggregate[i]].get());
merge(merged_columns, queue_without_collation);
for (auto & pos : converted_lc_columns)
{
auto & from_type = header.getByPosition(pos).type;
auto & to_type = result_header.getByPosition(pos).type;
merged_columns[pos] = (*recursiveTypeConversion(std::move(merged_columns[pos]), from_type, to_type)).mutate();
}
return result_header.cloneWithColumns(std::move(merged_columns));
2014-05-26 16:11:20 +00:00
}
2019-12-22 00:19:07 +00:00
void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue)
2014-05-26 16:11:20 +00:00
{
size_t merged_rows = 0;
/// We take the rows in the correct order and put them in `merged_block`, while the rows are no more than `max_block_size`
2019-12-22 00:19:07 +00:00
while (queue.isValid())
{
2019-12-22 00:19:07 +00:00
SortCursor current = queue.current();
setPrimaryKeyRef(next_key, current);
bool key_differs;
if (current_key.empty()) /// The first key encountered.
{
setPrimaryKeyRef(current_key, current);
key_differs = true;
}
else
key_differs = next_key != current_key;
/// if there are enough rows accumulated and the last one is calculated completely
2019-03-11 08:24:52 +00:00
if (key_differs && merged_rows >= max_block_size)
{
/// Write the simple aggregation result for the previous group.
insertSimpleAggregationResult(merged_columns);
return;
}
if (key_differs)
{
current_key.swap(next_key);
/// We will write the data for the group. We copy the values of ordinary columns.
for (size_t i = 0, size = column_numbers_not_to_aggregate.size(); i < size; ++i)
{
size_t j = column_numbers_not_to_aggregate[i];
merged_columns[j]->insertFrom(*current->all_columns[j], current->pos);
}
/// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function.
for (auto & column_to_aggregate : columns_to_aggregate)
column_to_aggregate->insertDefault();
/// Write the simple aggregation result for the previous group.
if (merged_rows > 0)
insertSimpleAggregationResult(merged_columns);
/// Reset simple aggregation states for next row
for (auto & desc : columns_to_simple_aggregate)
desc.createState();
if (allocatesMemoryInArena)
arena = std::make_unique<Arena>();
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
++merged_rows;
}
addRow(current);
if (!current->isLast())
{
2019-12-22 00:19:07 +00:00
queue.next();
}
else
{
/// We fetch the next block from the appropriate source, if there is one.
fetchNextBlock(current, queue);
}
}
/// Write the simple aggregation result for the previous group.
if (merged_rows > 0)
insertSimpleAggregationResult(merged_columns);
finished = true;
2014-05-26 16:11:20 +00:00
}
2016-04-14 05:03:33 +00:00
void AggregatingSortedBlockInputStream::addRow(SortCursor & cursor)
2016-04-14 05:03:33 +00:00
{
for (size_t i = 0, size = column_numbers_to_aggregate.size(); i < size; ++i)
{
size_t j = column_numbers_to_aggregate[i];
columns_to_aggregate[i]->insertMergeFrom(*cursor->all_columns[j], cursor->pos);
}
for (auto & desc : columns_to_simple_aggregate)
{
auto & col = cursor->all_columns[desc.column_number];
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
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get());
}
}
void AggregatingSortedBlockInputStream::insertSimpleAggregationResult(MutableColumns & merged_columns)
{
for (auto & desc : columns_to_simple_aggregate)
{
desc.function->insertResultInto(desc.state.data(), *merged_columns[desc.column_number]);
desc.destroyState();
}
2016-04-14 05:03:33 +00:00
}
2014-05-26 16:11:20 +00:00
}