2017-12-15 21:32:25 +00:00
|
|
|
#pragma once
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <AggregateFunctions/IAggregateFunction.h>
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/IColumn.h>
|
2019-10-07 18:56:03 +00:00
|
|
|
#include <Common/PODArray.h>
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Field.h>
|
2014-05-22 14:09:10 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadBufferFromString.h>
|
|
|
|
#include <IO/WriteBuffer.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2015-10-12 07:05:54 +00:00
|
|
|
|
2019-01-22 21:07:05 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2011-09-19 01:42:16 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-09-02 05:23:20 +00:00
|
|
|
class Arena;
|
|
|
|
using ArenaPtr = std::shared_ptr<Arena>;
|
2019-07-16 18:29:17 +00:00
|
|
|
using ConstArenaPtr = std::shared_ptr<const Arena>;
|
|
|
|
using ConstArenas = std::vector<ConstArenaPtr>;
|
2018-09-02 05:23:20 +00:00
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
|
2017-12-14 04:25:22 +00:00
|
|
|
/** Column of states of aggregate functions.
|
2017-03-09 00:56:38 +00:00
|
|
|
* 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
|
|
|
*/
|
2019-04-19 20:21:17 +00:00
|
|
|
class ColumnAggregateFunction final : public COWHelper<IColumn, ColumnAggregateFunction>
|
2014-06-05 20:34:13 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-12-15 21:32:25 +00:00
|
|
|
using Container = PaddedPODArray<AggregateDataPtr>;
|
2014-06-05 20:34:13 +00:00
|
|
|
|
2014-06-08 22:17:44 +00:00
|
|
|
private:
|
2019-04-19 20:21:17 +00:00
|
|
|
friend class COWHelper<IColumn, ColumnAggregateFunction>;
|
2017-12-14 04:25:22 +00:00
|
|
|
|
2019-07-16 18:29:17 +00:00
|
|
|
/// Arenas used by function states that are created elsewhere. We own these
|
|
|
|
/// arenas in the sense of extending their lifetime, but do not modify them.
|
|
|
|
/// Even reading these arenas is unsafe, because they may be shared with
|
|
|
|
/// other data blocks and modified by other threads concurrently.
|
|
|
|
ConstArenas foreign_arenas;
|
|
|
|
|
|
|
|
/// Arena for allocating the internals of function states created by current
|
|
|
|
/// column (e.g., when inserting new states).
|
|
|
|
ArenaPtr my_arena;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Used for destroying states and for finalization of values.
|
|
|
|
AggregateFunctionPtr func;
|
2014-06-05 20:34:13 +00:00
|
|
|
|
2017-04-01 07:20:54 +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.
|
2017-12-13 01:27:53 +00:00
|
|
|
ColumnPtr src;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Array of pointers to aggregation states, that are placed in arenas.
|
2017-12-15 21:32:25 +00:00
|
|
|
Container data;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-12-13 19:14:12 +00:00
|
|
|
ColumnAggregateFunction() {}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Create a new column that has another column as a source.
|
2019-07-16 18:29:17 +00:00
|
|
|
MutablePtr createView() const;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
/// If we have another column as a source (owner of data), copy all data to ourself and reset source.
|
|
|
|
/// This is needed before inserting new elements, because we must own these elements (to destroy them in destructor),
|
|
|
|
/// but ownership of different elements cannot be mixed by different columns.
|
|
|
|
void ensureOwnership();
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ColumnAggregateFunction(const AggregateFunctionPtr & func_)
|
|
|
|
: func(func_)
|
|
|
|
{
|
|
|
|
}
|
2014-06-06 19:35:41 +00:00
|
|
|
|
2019-07-16 18:29:17 +00:00
|
|
|
ColumnAggregateFunction(const AggregateFunctionPtr & func_,
|
|
|
|
const ConstArenas & arenas_)
|
|
|
|
: foreign_arenas(arenas_), func(func_)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
}
|
2016-05-28 17:31:50 +00:00
|
|
|
|
2019-07-16 18:29:17 +00:00
|
|
|
|
|
|
|
ColumnAggregateFunction(const ColumnAggregateFunction & src_);
|
2017-12-14 03:56:56 +00:00
|
|
|
|
2019-02-11 13:11:52 +00:00
|
|
|
String getTypeString() const;
|
|
|
|
|
2017-12-13 19:14:12 +00:00
|
|
|
public:
|
2018-06-03 17:43:56 +00:00
|
|
|
~ColumnAggregateFunction() override;
|
2013-02-08 23:41:05 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void set(const AggregateFunctionPtr & func_)
|
|
|
|
{
|
|
|
|
func = func_;
|
|
|
|
}
|
2013-02-08 20:34:30 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
AggregateFunctionPtr getAggregateFunction() { return func; }
|
|
|
|
AggregateFunctionPtr getAggregateFunction() const { return func; }
|
2014-02-27 12:49:21 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Take shared ownership of Arena, that holds memory for states of aggregate functions.
|
2019-07-16 18:29:17 +00:00
|
|
|
void addArena(ConstArenaPtr arena_);
|
2013-02-08 23:41:05 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Transform column with states of aggregate functions to column with final result values.
|
|
|
|
*/
|
2017-12-13 01:27:53 +00:00
|
|
|
MutableColumnPtr convertToValues() const;
|
2014-02-27 12:49:21 +00:00
|
|
|
|
2017-12-07 22:11:51 +00:00
|
|
|
std::string getName() const override { return "AggregateFunction(" + func->getName() + ")"; }
|
|
|
|
const char * getFamilyName() const override { return "AggregateFunction"; }
|
2014-06-05 20:34:13 +00:00
|
|
|
|
2019-05-13 13:49:53 +00:00
|
|
|
bool tryFinalizeAggregateFunction(MutableColumnPtr* res_) const;
|
2019-04-21 14:32:42 +00:00
|
|
|
MutableColumnPtr predictValues(Block & block, const ColumnNumbers & arguments, const Context & context) const;
|
2019-01-22 21:07:05 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size() const override
|
|
|
|
{
|
|
|
|
return getData().size();
|
|
|
|
}
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2017-12-13 01:27:53 +00:00
|
|
|
MutableColumnPtr cloneEmpty() const override;
|
2011-09-19 01:42:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Field operator[](size_t n) const override;
|
2012-10-07 06:30:10 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void get(size_t n, Field & res) const override;
|
2013-01-07 06:47:15 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
StringRef getDataAt(size_t n) const override;
|
2014-05-22 14:09:10 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void insertData(const char * pos, size_t length) override;
|
2014-05-26 16:11:20 +00:00
|
|
|
|
2018-08-27 18:16:32 +00:00
|
|
|
void insertFrom(const IColumn & from, size_t n) override;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void insertFrom(ConstAggregateDataPtr place);
|
2017-02-15 11:23:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Merge state at last row with specified state in another column.
|
|
|
|
void insertMergeFrom(ConstAggregateDataPtr place);
|
2016-02-16 16:39:39 +00:00
|
|
|
|
2018-08-27 18:16:32 +00:00
|
|
|
void insertMergeFrom(const IColumn & from, size_t n);
|
2015-11-10 20:39:11 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Arena & createOrGetArena();
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void insert(const Field & x) override;
|
2016-04-14 05:03:33 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void insertDefault() override;
|
2016-04-14 05:03:33 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
|
2015-10-04 03:17:36 +00:00
|
|
|
|
2020-03-09 01:03:43 +00:00
|
|
|
const char * deserializeAndInsertFromArena(const char * src_arena) override;
|
2015-10-04 03:17:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void updateHashWithValue(size_t n, SipHash & hash) const override;
|
2016-07-10 15:58:58 +00:00
|
|
|
|
2020-03-18 16:03:55 +00:00
|
|
|
void updateWeakHash32(WeakHash32 & hash) const override;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t byteSize() const override;
|
2014-05-28 14:54:42 +00:00
|
|
|
|
2017-07-13 16:49:09 +00:00
|
|
|
size_t allocatedBytes() const override;
|
2017-01-17 20:54:32 +00:00
|
|
|
|
2019-03-10 03:16:51 +00:00
|
|
|
void protect() override;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void insertRangeFrom(const IColumn & from, size_t start, size_t length) override;
|
2013-05-03 05:23:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void popBack(size_t n) override;
|
2016-02-16 16:39:39 +00:00
|
|
|
|
2018-03-20 14:17:09 +00:00
|
|
|
ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override;
|
2015-12-05 07:01:18 +00:00
|
|
|
|
2019-02-18 19:44:26 +00:00
|
|
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
2013-05-03 05:23:14 +00:00
|
|
|
|
2019-02-18 17:28:53 +00:00
|
|
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
2018-04-18 21:00:47 +00:00
|
|
|
|
|
|
|
template <typename Type>
|
2019-02-18 17:28:53 +00:00
|
|
|
ColumnPtr indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const;
|
2018-04-18 21:00:47 +00:00
|
|
|
|
2018-03-20 14:17:09 +00:00
|
|
|
ColumnPtr replicate(const Offsets & offsets) const override;
|
2017-02-11 20:20:57 +00:00
|
|
|
|
2017-12-13 19:14:12 +00:00
|
|
|
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
2017-02-11 20:20:57 +00:00
|
|
|
|
2017-07-06 13:54:55 +00:00
|
|
|
void gather(ColumnGathererStream & gatherer_stream) override;
|
|
|
|
|
2017-12-01 19:34:51 +00:00
|
|
|
int compareAt(size_t, size_t, const IColumn &, int) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return 0;
|
|
|
|
}
|
2011-09-26 11:05:38 +00:00
|
|
|
|
2019-02-18 19:44:26 +00:00
|
|
|
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** More efficient manipulation methods */
|
2017-12-15 21:32:25 +00:00
|
|
|
Container & getData()
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return data;
|
|
|
|
}
|
2014-06-05 23:52:28 +00:00
|
|
|
|
2017-12-15 21:32:25 +00:00
|
|
|
const Container & getData() const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return data;
|
|
|
|
}
|
2016-07-05 16:23:37 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void getExtremes(Field & min, Field & max) const override;
|
2020-02-26 18:32:15 +00:00
|
|
|
|
|
|
|
bool structureEquals(const IColumn &) const override;
|
2011-09-19 01:42:16 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
}
|