Added aggregate functions groupBitAnd, groupBitOr, groupBitXor [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-08-30 21:52:14 +03:00
parent 495daffa16
commit 1531268281
6 changed files with 145 additions and 0 deletions

View File

@ -0,0 +1,39 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionBitwise.h>
#include <AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res(createWithUnsignedIntegerType<AggregateFunctionBitwise, Data>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory)
{
factory.registerFunction("groupBitOr", createAggregateFunctionBitwise<AggregateFunctionGroupBitOrData>);
factory.registerFunction("groupBitAnd", createAggregateFunctionBitwise<AggregateFunctionGroupBitAndData>);
factory.registerFunction("groupBitXor", createAggregateFunctionBitwise<AggregateFunctionGroupBitXorData>);
/// Aliases for compatibility with MySQL.
factory.registerFunction("BIT_OR", createAggregateFunctionBitwise<AggregateFunctionGroupBitOrData>, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("BIT_AND", createAggregateFunctionBitwise<AggregateFunctionGroupBitAndData>, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("BIT_XOR", createAggregateFunctionBitwise<AggregateFunctionGroupBitXorData>, AggregateFunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,87 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <AggregateFunctions/IUnaryAggregateFunction.h>
namespace DB
{
template <typename T>
struct AggregateFunctionGroupBitOrData
{
T value = 0;
static const char * name() { return "groupBitOr"; }
void update(T x) { value |= x; }
};
template <typename T>
struct AggregateFunctionGroupBitAndData
{
T value = -1; /// Two's complement arithmetic, sign extension.
static const char * name() { return "groupBitAnd"; }
void update(T x) { value &= x; }
};
template <typename T>
struct AggregateFunctionGroupBitXorData
{
T value = 0;
static const char * name() { return "groupBitXor"; }
void update(T x) { value ^= x; }
};
/// Counts bitwise operation on numbers.
template <typename T, typename Data>
class AggregateFunctionBitwise final : public IUnaryAggregateFunction<Data, AggregateFunctionBitwise<T, Data>>
{
public:
String getName() const override { return Data::name(); }
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeNumber<T>>();
}
void setArgument(const DataTypePtr & argument)
{
if (!argument->behavesAsNumber())
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena *) const
{
this->data(place).update(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).update(this->data(rhs).value);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
writeBinary(this->data(place).value, buf);
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
readBinary(this->data(place).value, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnVector<T> &>(to).getData().push_back(this->data(place).value);
}
};
}

View File

@ -92,6 +92,18 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
}
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data>
static IAggregateFunction * createWithUnsignedIntegerType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data<UInt8>>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<UInt16, Data<UInt16>>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<UInt32, Data<UInt32>>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<UInt64, Data<UInt64>>;
else
return nullptr;
}
/** For template with two arguments.
*/
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate>

View File

@ -23,6 +23,7 @@ void registerAggregateFunctionSum(AggregateFunctionFactory & factory);
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory);
void registerAggregateFunctionTopK(AggregateFunctionFactory & factory);
void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory);
void registerAggregateFunctionDebug(AggregateFunctionFactory & factory);
@ -48,6 +49,7 @@ void registerAggregateFunctions()
registerAggregateFunctionsUniq(factory);
registerAggregateFunctionUniqUpTo(factory);
registerAggregateFunctionTopK(factory);
registerAggregateFunctionsBitwise(factory);
registerAggregateFunctionDebug(factory);
}

View File

@ -0,0 +1,4 @@
0 [0,4,8,12,16] 28 0 16
1 [1,5,9,13,17] 29 1 17
2 [2,6,10,14,18] 30 2 18
3 [3,7,11,15,19] 31 3 19

View File

@ -0,0 +1 @@
SELECT number % 4 AS k, groupArray(number), groupBitOr(number), groupBitAnd(number), groupBitXor(number) FROM (SELECT * FROM system.numbers LIMIT 20) GROUP BY k ORDER BY k;