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: 8f8d2c04 ("Merge pull request #4629 from bgranvea/simple_aggregate_function")
上级 7b34c1c4
#include <DataStreams/AggregatingSortedBlockInputStream.h>
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Arena.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
......@@ -134,6 +135,8 @@ void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, s
for (auto & desc : columns_to_simple_aggregate)
desc.createState();
arena = std::make_unique<Arena>();
++merged_rows;
}
......@@ -169,7 +172,7 @@ void AggregatingSortedBlockInputStream::addRow(SortCursor & cursor)
for (auto & desc : columns_to_simple_aggregate)
{
auto & col = cursor->all_columns[desc.column_number];
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr);
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get());
}
}
......
#pragma once
#include <common/logger_useful.h>
#include <memory>
#include <Core/Row.h>
#include <Core/ColumnNumbers.h>
......@@ -13,6 +14,8 @@
namespace DB
{
class Arena;
/** 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,
......@@ -63,6 +66,9 @@ private:
*/
void insertSimpleAggregationResult(MutableColumns & merged_columns);
/// Memory pool for SimpleAggregateFunction.
std::unique_ptr<Arena> arena;
/// Stores information for aggregation of SimpleAggregateFunction columns
struct SimpleAggregateDescription
{
......
......@@ -40,4 +40,5 @@ SimpleAggregateFunction(sum, Float64)
8 16
9 18
1 1 2 2.2.2.2
10 2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222 20 20.20.20.20
SimpleAggregateFunction(anyLast, Nullable(String)) SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) SimpleAggregateFunction(anyLast, IPv4)
......@@ -22,6 +22,9 @@ drop table if exists test.simple;
create table test.simple (id UInt64,nullable_str SimpleAggregateFunction(anyLast,Nullable(String)),low_str SimpleAggregateFunction(anyLast,LowCardinality(Nullable(String))),ip SimpleAggregateFunction(anyLast,IPv4)) engine=AggregatingMergeTree order by id;
insert into test.simple values(1,'1','1','1.1.1.1');
insert into test.simple values(1,null,'2','2.2.2.2');
-- String longer then MAX_SMALL_STRING_SIZE (actual string length is 100)
insert into test.simple values(10,'10','10','10.10.10.10');
insert into test.simple values(10,'2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222','20','20.20.20.20');
select * from test.simple final;
select toTypeName(nullable_str),toTypeName(low_str),toTypeName(ip) from test.simple limit 1;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册