2011-09-19 01:42:16 +00:00
|
|
|
#pragma once
|
|
|
|
|
2013-02-08 19:34:44 +00:00
|
|
|
#include <DB/Common/Arena.h>
|
|
|
|
|
2011-09-19 01:42:16 +00:00
|
|
|
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
|
|
|
|
2014-06-05 23:52:28 +00:00
|
|
|
#include <DB/Columns/IColumn.h>
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2014-05-22 14:09:10 +00:00
|
|
|
#include <DB/Core/Field.h>
|
|
|
|
|
|
|
|
#include <DB/IO/ReadBufferFromString.h>
|
2015-10-12 07:05:54 +00:00
|
|
|
#include <DB/IO/WriteBuffer.h>
|
|
|
|
#include <DB/IO/WriteHelpers.h>
|
|
|
|
|
2011-09-19 01:42:16 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int PARAMETER_OUT_OF_BOUND;
|
|
|
|
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
|
|
|
}
|
|
|
|
|
2013-02-08 23:41:05 +00:00
|
|
|
|
2017-03-09 00:56:38 +00:00
|
|
|
/** State column of aggregate functions.
|
|
|
|
* Presented as an array of pointers to the states of aggregate functions (data).
|
|
|
|
* The states themselves are stored in one of the pools (arenas).
|
2014-06-08 22:17:44 +00:00
|
|
|
*
|
2017-03-09 00:56:38 +00:00
|
|
|
* It can be in two variants:
|
2014-06-08 22:17:44 +00:00
|
|
|
*
|
2017-03-09 00:56:38 +00:00
|
|
|
* 1. Own its values - that is, be responsible for destroying them.
|
2017-03-09 04:18:41 +00:00
|
|
|
* The column consists of the values "assigned to it" after the aggregation is performed (see Aggregator, convertToBlocks function),
|
|
|
|
* or from values created by itself (see `insert` method).
|
2017-03-09 00:56:38 +00:00
|
|
|
* In this case, `src` will be `nullptr`, and the column itself will be destroyed (call `IAggregateFunction::destroy`)
|
|
|
|
* states of aggregate functions in the destructor.
|
2014-06-05 20:34:13 +00:00
|
|
|
*
|
2017-03-09 04:18:41 +00:00
|
|
|
* 2. Do not own its values, but use values taken from another ColumnAggregateFunction column.
|
2017-03-09 00:56:38 +00:00
|
|
|
* For example, this is a column obtained by permutation/filtering or other transformations from another column.
|
2017-03-09 04:18:41 +00:00
|
|
|
* In this case, `src` will be `shared ptr` to the source column. Destruction of values will be handled by this source column.
|
2014-06-08 22:17:44 +00:00
|
|
|
*
|
2017-03-09 00:56:38 +00:00
|
|
|
* This solution is somewhat limited:
|
2017-03-09 04:18:41 +00:00
|
|
|
* - the variant in which the column contains a part of "it's own" and a part of "another's" values is not supported;
|
2017-03-09 00:56:38 +00:00
|
|
|
* - the option of having multiple source columns is not supported, which may be necessary for a more optimal merge of the two columns.
|
2014-06-08 22:17:44 +00:00
|
|
|
*
|
2017-03-09 00:56:38 +00:00
|
|
|
* These restrictions can be removed if you add an array of flags or even refcount,
|
2017-03-09 04:18:41 +00:00
|
|
|
* specifying which individual values should be destroyed and which ones should not.
|
2017-03-09 00:56:38 +00:00
|
|
|
* Clearly, this method would have a substantially non-zero price.
|
2014-06-05 20:34:13 +00:00
|
|
|
*/
|
2016-05-28 17:31:50 +00:00
|
|
|
class ColumnAggregateFunction final : public IColumn, public std::enable_shared_from_this<ColumnAggregateFunction>
|
2014-06-05 20:34:13 +00:00
|
|
|
{
|
|
|
|
public:
|
2016-04-15 00:33:21 +00:00
|
|
|
using Container_t = PaddedPODArray<AggregateDataPtr>;
|
2014-06-05 20:34:13 +00:00
|
|
|
|
2014-06-08 22:17:44 +00:00
|
|
|
private:
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Memory pools. Aggregate states are allocated from them.
|
2016-05-28 17:31:50 +00:00
|
|
|
Arenas arenas;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Used for destroying states and for finalization of values.
|
2016-05-28 17:31:50 +00:00
|
|
|
AggregateFunctionPtr func;
|
2014-06-05 20:34:13 +00:00
|
|
|
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Source column. Used (holds source from destruction),
|
|
|
|
/// if this column has been constructed from another and uses all or part of its values.
|
2016-06-21 21:42:57 +00:00
|
|
|
std::shared_ptr<const ColumnAggregateFunction> src;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Array of pointers to aggregation states, that are placed in arenas.
|
2016-05-28 17:31:50 +00:00
|
|
|
Container_t data;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2016-05-28 05:31:36 +00:00
|
|
|
public:
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Create a new column that has another column as a source.
|
2016-05-28 17:31:50 +00:00
|
|
|
ColumnAggregateFunction(const ColumnAggregateFunction & other)
|
|
|
|
: arenas(other.arenas), func(other.func), src(other.shared_from_this())
|
2014-06-08 22:17:44 +00:00
|
|
|
{
|
|
|
|
}
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2014-06-08 22:17:44 +00:00
|
|
|
ColumnAggregateFunction(const AggregateFunctionPtr & func_)
|
2016-05-28 17:31:50 +00:00
|
|
|
: func(func_)
|
2014-06-06 19:35:41 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2014-06-08 22:17:44 +00:00
|
|
|
ColumnAggregateFunction(const AggregateFunctionPtr & func_, const Arenas & arenas_)
|
2016-05-28 17:31:50 +00:00
|
|
|
: arenas(arenas_), func(func_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2017-03-09 04:26:17 +00:00
|
|
|
~ColumnAggregateFunction()
|
2013-02-08 23:41:05 +00:00
|
|
|
{
|
2016-06-09 04:54:30 +00:00
|
|
|
if (!func->hasTrivialDestructor() && !src)
|
2016-05-28 17:31:50 +00:00
|
|
|
for (auto val : data)
|
|
|
|
func->destroy(val);
|
2013-02-08 23:41:05 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 04:26:17 +00:00
|
|
|
void set(const AggregateFunctionPtr & func_)
|
2013-02-08 19:34:44 +00:00
|
|
|
{
|
2016-05-28 17:31:50 +00:00
|
|
|
func = func_;
|
2013-02-08 20:34:30 +00:00
|
|
|
}
|
|
|
|
|
2016-05-28 17:31:50 +00:00
|
|
|
AggregateFunctionPtr getAggregateFunction() { return func; }
|
|
|
|
AggregateFunctionPtr getAggregateFunction() const { return func; }
|
2014-02-27 12:49:21 +00:00
|
|
|
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Take shared ownership of Arena, that holds memory for states of aggregate functions.
|
2013-02-08 23:41:05 +00:00
|
|
|
void addArena(ArenaPtr arena_)
|
2013-02-08 20:34:30 +00:00
|
|
|
{
|
2014-06-08 22:17:44 +00:00
|
|
|
arenas.push_back(arena_);
|
2013-02-08 19:34:44 +00:00
|
|
|
}
|
2013-02-08 23:41:05 +00:00
|
|
|
|
2016-11-01 21:06:41 +00:00
|
|
|
/** Transform column with states of aggregate functions to column with final result values.
|
2015-11-11 01:28:34 +00:00
|
|
|
*/
|
|
|
|
ColumnPtr convertToValues() const;
|
2014-02-27 12:49:21 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
std::string getName() const override { return "ColumnAggregateFunction"; }
|
2014-06-05 20:34:13 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
size_t sizeOfField() const override { return sizeof(getData()[0]); }
|
2014-06-05 19:52:13 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
size_t size() const override
|
2014-06-05 23:52:28 +00:00
|
|
|
{
|
2014-06-08 22:17:44 +00:00
|
|
|
return getData().size();
|
2013-02-03 23:11:21 +00:00
|
|
|
}
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
ColumnPtr cloneEmpty() const override
|
|
|
|
{
|
2016-05-28 17:31:50 +00:00
|
|
|
return std::make_shared<ColumnAggregateFunction>(func, Arenas(1, std::make_shared<Arena>()));
|
2014-06-05 23:52:28 +00:00
|
|
|
};
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
Field operator[](size_t n) const override
|
2011-09-19 01:42:16 +00:00
|
|
|
{
|
2014-05-22 14:09:10 +00:00
|
|
|
Field field = String();
|
|
|
|
{
|
|
|
|
WriteBufferFromString buffer(field.get<String &>());
|
2016-05-28 17:31:50 +00:00
|
|
|
func->serialize(getData()[n], buffer);
|
2014-05-22 14:09:10 +00:00
|
|
|
}
|
|
|
|
return field;
|
2011-09-19 01:42:16 +00:00
|
|
|
}
|
2012-10-07 06:30:10 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
void get(size_t n, Field & res) const override
|
2013-01-07 06:47:15 +00:00
|
|
|
{
|
2014-06-05 23:52:28 +00:00
|
|
|
res = String();
|
2014-05-22 14:09:10 +00:00
|
|
|
{
|
|
|
|
WriteBufferFromString buffer(res.get<String &>());
|
2016-05-28 17:31:50 +00:00
|
|
|
func->serialize(getData()[n], buffer);
|
2014-05-22 14:09:10 +00:00
|
|
|
}
|
2013-01-07 06:47:15 +00:00
|
|
|
}
|
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
StringRef getDataAt(size_t n) const override
|
2012-10-07 06:30:10 +00:00
|
|
|
{
|
2014-06-08 22:17:44 +00:00
|
|
|
return StringRef(reinterpret_cast<const char *>(&getData()[n]), sizeof(getData()[n]));
|
2014-05-22 14:09:10 +00:00
|
|
|
}
|
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
void insertData(const char * pos, size_t length) override
|
2014-05-22 14:09:10 +00:00
|
|
|
{
|
2014-06-08 22:17:44 +00:00
|
|
|
getData().push_back(*reinterpret_cast<const AggregateDataPtr *>(pos));
|
2014-05-26 16:11:20 +00:00
|
|
|
}
|
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
void insertFrom(const IColumn & src, size_t n) override
|
2014-05-26 16:11:20 +00:00
|
|
|
{
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Must create new state of aggregate function and take ownership of it,
|
|
|
|
/// because ownership of states of aggregate function cannot be shared for individual rows,
|
|
|
|
/// (only as a whole, see comment above).
|
|
|
|
insertDefault();
|
|
|
|
insertMergeFrom(src, n);
|
2014-06-08 22:17:44 +00:00
|
|
|
}
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-02-15 11:23:38 +00:00
|
|
|
void insertFrom(ConstAggregateDataPtr place)
|
|
|
|
{
|
|
|
|
insertDefault();
|
|
|
|
insertMergeFrom(place);
|
|
|
|
}
|
|
|
|
|
2016-11-01 21:06:41 +00:00
|
|
|
/// Merge state at last row with specified state in another column.
|
2017-02-15 11:23:38 +00:00
|
|
|
void insertMergeFrom(ConstAggregateDataPtr place)
|
|
|
|
{
|
|
|
|
func->merge(getData().back(), place, &createOrGetArena());
|
|
|
|
}
|
|
|
|
|
2014-06-08 22:17:44 +00:00
|
|
|
void insertMergeFrom(const IColumn & src, size_t n)
|
|
|
|
{
|
2017-02-15 11:23:38 +00:00
|
|
|
insertMergeFrom(static_cast<const ColumnAggregateFunction &>(src).getData()[n]);
|
2014-06-05 23:52:28 +00:00
|
|
|
}
|
|
|
|
|
2016-02-16 16:39:39 +00:00
|
|
|
Arena & createOrGetArena()
|
|
|
|
{
|
|
|
|
if (unlikely(arenas.empty()))
|
2016-05-28 10:15:36 +00:00
|
|
|
arenas.emplace_back(std::make_shared<Arena>());
|
2016-02-16 16:39:39 +00:00
|
|
|
return *arenas.back().get();
|
|
|
|
}
|
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
void insert(const Field & x) override
|
2014-06-05 23:52:28 +00:00
|
|
|
{
|
2016-05-28 17:31:50 +00:00
|
|
|
IAggregateFunction * function = func.get();
|
2014-06-08 22:17:44 +00:00
|
|
|
|
2016-02-16 16:39:39 +00:00
|
|
|
Arena & arena = createOrGetArena();
|
2015-11-10 20:39:11 +00:00
|
|
|
|
2016-02-16 16:39:39 +00:00
|
|
|
getData().push_back(arena.alloc(function->sizeOfData()));
|
2014-06-08 22:17:44 +00:00
|
|
|
function->create(getData().back());
|
|
|
|
ReadBufferFromString read_buffer(x.get<const String &>());
|
2016-09-22 23:26:08 +00:00
|
|
|
function->deserialize(getData().back(), read_buffer, &arena);
|
2014-06-05 23:52:28 +00:00
|
|
|
}
|
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
void insertDefault() override
|
2014-06-05 23:52:28 +00:00
|
|
|
{
|
2016-05-28 17:31:50 +00:00
|
|
|
IAggregateFunction * function = func.get();
|
2016-04-14 05:03:33 +00:00
|
|
|
|
|
|
|
Arena & arena = createOrGetArena();
|
|
|
|
|
|
|
|
getData().push_back(arena.alloc(function->sizeOfData()));
|
|
|
|
function->create(getData().back());
|
2014-06-05 23:52:28 +00:00
|
|
|
}
|
|
|
|
|
2015-10-04 03:17:36 +00:00
|
|
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
|
|
|
{
|
|
|
|
throw Exception("Method serializeValueIntoArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
|
|
|
|
|
|
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
|
|
|
{
|
|
|
|
throw Exception("Method deserializeAndInsertFromArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
|
|
|
|
2017-02-16 22:05:48 +00:00
|
|
|
void updateHashWithValue(size_t n, SipHash & hash) const override;
|
2016-07-10 15:58:58 +00:00
|
|
|
|
2017-01-27 12:38:59 +00:00
|
|
|
size_t byteSize() const override;
|
2014-05-28 14:54:42 +00:00
|
|
|
|
2017-01-27 12:38:59 +00:00
|
|
|
size_t allocatedSize() const override;
|
2017-01-17 20:54:32 +00:00
|
|
|
|
2017-01-20 05:00:04 +00:00
|
|
|
void insertRangeFrom(const IColumn & from, size_t start, size_t length) override;
|
2013-05-03 05:23:14 +00:00
|
|
|
|
2016-02-16 16:39:39 +00:00
|
|
|
void popBack(size_t n) override
|
|
|
|
{
|
2016-05-28 17:31:50 +00:00
|
|
|
size_t size = data.size();
|
|
|
|
size_t new_size = size - n;
|
|
|
|
|
|
|
|
if (!src)
|
|
|
|
for (size_t i = new_size; i < size; ++i)
|
|
|
|
func->destroy(data[i]);
|
|
|
|
|
|
|
|
data.resize_assume_reserved(new_size);
|
2016-02-16 16:39:39 +00:00
|
|
|
}
|
|
|
|
|
2017-01-20 05:00:04 +00:00
|
|
|
ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override;
|
2015-12-05 07:01:18 +00:00
|
|
|
|
2017-01-20 05:00:04 +00:00
|
|
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
2013-05-03 05:23:14 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
ColumnPtr replicate(const Offsets_t & offsets) const override
|
2013-02-08 23:41:05 +00:00
|
|
|
{
|
|
|
|
throw Exception("Method replicate is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
|
|
|
|
2017-02-11 20:20:57 +00:00
|
|
|
Columns scatter(ColumnIndex num_columns, const Selector & selector) const override
|
|
|
|
{
|
|
|
|
/// Columns with scattered values will point to this column as the owner of values.
|
|
|
|
Columns columns(num_columns);
|
|
|
|
for (auto & column : columns)
|
|
|
|
column = std::make_shared<ColumnAggregateFunction>(*this);
|
|
|
|
|
|
|
|
size_t num_rows = size();
|
|
|
|
|
|
|
|
{
|
|
|
|
size_t reserve_size = num_rows / num_columns * 1.1; /// 1.1 is just a guess. Better to use n-sigma rule.
|
|
|
|
|
|
|
|
if (reserve_size > 1)
|
|
|
|
for (auto & column : columns)
|
|
|
|
column->reserve(reserve_size);
|
|
|
|
}
|
|
|
|
|
|
|
|
for (size_t i = 0; i < num_rows; ++i)
|
|
|
|
static_cast<ColumnAggregateFunction &>(*columns[selector[i]]).data.push_back(data[i]);
|
|
|
|
|
|
|
|
return columns;
|
|
|
|
}
|
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
|
2011-09-19 01:42:16 +00:00
|
|
|
{
|
|
|
|
return 0;
|
|
|
|
}
|
2011-09-26 11:05:38 +00:00
|
|
|
|
2014-10-31 22:15:17 +00:00
|
|
|
void getPermutation(bool reverse, size_t limit, Permutation & res) const override
|
2011-09-26 11:05:38 +00:00
|
|
|
{
|
2014-06-08 22:17:44 +00:00
|
|
|
size_t s = getData().size();
|
2013-12-08 02:29:40 +00:00
|
|
|
res.resize(s);
|
2011-09-26 11:05:38 +00:00
|
|
|
for (size_t i = 0; i < s; ++i)
|
|
|
|
res[i] = i;
|
|
|
|
}
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-03-09 04:26:17 +00:00
|
|
|
/** More efficient manipulation methods */
|
2014-06-05 23:52:28 +00:00
|
|
|
Container_t & getData()
|
|
|
|
{
|
2016-05-28 17:31:50 +00:00
|
|
|
return data;
|
2014-06-05 23:52:28 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
const Container_t & getData() const
|
|
|
|
{
|
2016-05-28 17:31:50 +00:00
|
|
|
return data;
|
2014-06-05 23:52:28 +00:00
|
|
|
}
|
2016-07-05 16:23:37 +00:00
|
|
|
|
2016-08-10 19:12:29 +00:00
|
|
|
void getExtremes(Field & min, Field & max) const override
|
2016-07-05 16:23:37 +00:00
|
|
|
{
|
|
|
|
throw Exception("Method getExtremes is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
2011-09-19 01:42:16 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
}
|