ClickHouse/dbms/include/DB/AggregateFunctions/IAggregateFunction.h

164 lines
5.5 KiB
C++
Raw Normal View History

2011-09-19 03:40:05 +00:00
#pragma once
2011-12-19 08:06:31 +00:00
#include <memory>
2011-09-19 03:40:05 +00:00
#include <DB/Core/Row.h>
#include <DB/DataTypes/IDataType.h>
#include <DB/Common/typeid_cast.h>
#include <DB/Common/Arena.h>
2011-09-19 03:40:05 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2016-01-12 02:21:15 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
}
using AggregateDataPtr = char *;
using ConstAggregateDataPtr = const char *;
2017-03-09 00:56:38 +00:00
/** Aggregate functions interface.
* Instances of classes with this interface do not contain the data itself for aggregation,
* but contain only metadata (description) of the aggregate function,
* as well as methods for creating, deleting and working with data.
* The data resulting from the aggregation (intermediate computing states) is stored in other objects
* (which can be created in some pool),
* and IAggregateFunction is the external interface for manipulating them.
2011-09-19 03:40:05 +00:00
*/
class IAggregateFunction
{
public:
2017-03-09 00:56:38 +00:00
/// Get main function name.
2011-09-19 03:40:05 +00:00
virtual String getName() const = 0;
2017-03-09 04:26:17 +00:00
/** Specify the types of arguments. If the function does not apply to these arguments throw an exception.
* You must call before other calls.
2011-09-19 03:40:05 +00:00
*/
virtual void setArguments(const DataTypes & arguments) = 0;
2017-03-09 04:26:17 +00:00
/** Specify parameters for parametric aggregate functions.
* If no parameters are provided, or the passed parameters are not valid, throw an exception.
* If there are parameters - it is necessary to call before other calls, otherwise - do not call.
*/
virtual void setParameters(const Array & params)
{
throw Exception("Aggregate function " + getName() + " doesn't allow parameters.",
ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS);
}
2017-03-09 00:56:38 +00:00
/// Get the result type.
2011-09-19 03:40:05 +00:00
virtual DataTypePtr getReturnType() const = 0;
virtual ~IAggregateFunction() {};
2017-03-09 04:26:17 +00:00
/** Data functions. */
2017-03-09 04:26:17 +00:00
/** Create empty data for aggregation with `placement new` at the specified location.
* You will have to destroy them using the `destroy` method.
*/
virtual void create(AggregateDataPtr place) const = 0;
2017-03-09 04:26:17 +00:00
/// Delete data for aggregation.
2014-05-23 00:35:14 +00:00
virtual void destroy(AggregateDataPtr place) const noexcept = 0;
2017-03-09 00:56:38 +00:00
/// It is not necessary to delete data.
virtual bool hasTrivialDestructor() const = 0;
2017-03-09 04:26:17 +00:00
/// Get `sizeof` of structure with data.
virtual size_t sizeOfData() const = 0;
2017-03-09 04:26:17 +00:00
/// How the data structure should be aligned. NOTE: Currently not used (structures with aggregation state are put without alignment).
virtual size_t alignOfData() const = 0;
2016-09-22 23:26:08 +00:00
/** Adds a value into aggregation data on which place points to.
* columns points to columns containing arguments of aggregation function.
* row_num is number of row which should be added.
* Additional parameter arena should be used instead of standard memory allocator if the addition requires memory allocation.
*/
virtual void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const = 0;
2011-09-19 03:40:05 +00:00
2016-09-22 23:26:08 +00:00
/// Merges state (on which place points to) with other state of current aggregation function.
virtual void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const = 0;
2011-09-19 03:40:05 +00:00
2016-09-22 23:26:08 +00:00
/// Serializes state (to transmit it over the network, for example).
virtual void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const = 0;
2011-09-19 03:40:05 +00:00
2016-09-22 23:26:08 +00:00
/// Deserializes state. This function is called only for empty (just created) states.
virtual void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const = 0;
2011-09-19 03:40:05 +00:00
/// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()).
virtual bool allocatesMemoryInArena() const
{
return false;
}
2016-09-22 23:26:08 +00:00
/// Inserts results into a column.
virtual void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const = 0;
2014-05-21 13:27:40 +00:00
2017-03-09 04:26:17 +00:00
/** Returns true for aggregate functions of type -State.
2017-03-09 00:56:38 +00:00
* They are executed as other aggregate functions, but not finalized (return an aggregation state that can be combined with another).
2014-06-03 20:29:04 +00:00
*/
virtual bool isState() const { return false; }
2017-03-09 04:26:17 +00:00
/** The inner loop that uses the function pointer is better than using the virtual function.
* The reason is that in the case of virtual functions GCC 5.1.2 generates code,
* which, at each iteration of the loop, reloads the function address (the offset value in the virtual function table) from memory to the register.
* This gives a performance drop on simple queries around 12%.
2017-03-09 00:56:38 +00:00
* After the appearance of better compilers, the code can be removed.
*/
using AddFunc = void (*)(const IAggregateFunction *, AggregateDataPtr, const IColumn **, size_t, Arena *);
virtual AddFunc getAddressOfAddFunction() const = 0;
};
2017-03-09 00:56:38 +00:00
/// Implements several methods. T - type of structure with data for aggregation.
template <typename T>
class IAggregateFunctionHelper : public IAggregateFunction
{
protected:
using Data = T;
static Data & data(AggregateDataPtr place) { return *reinterpret_cast<Data*>(place); }
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data*>(place); }
public:
2015-11-11 02:04:23 +00:00
void create(AggregateDataPtr place) const override
{
new (place) Data;
}
2015-11-11 02:04:23 +00:00
void destroy(AggregateDataPtr place) const noexcept override
{
data(place).~Data();
}
2015-11-11 02:04:23 +00:00
bool hasTrivialDestructor() const override
{
return __has_trivial_destructor(Data);
}
2015-11-11 02:04:23 +00:00
size_t sizeOfData() const override
{
return sizeof(Data);
}
2017-03-09 00:56:38 +00:00
/// NOTE: Currently not used (structures with aggregation state are put without alignment).
2015-11-11 02:04:23 +00:00
size_t alignOfData() const override
{
return __alignof__(Data);
}
2011-09-19 03:40:05 +00:00
};
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
2011-09-19 03:40:05 +00:00
}