mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Added aggregate functions groupBitAnd, groupBitOr, groupBitXor [#CLICKHOUSE-2].
This commit is contained in:
parent
495daffa16
commit
1531268281
39
dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp
Normal file
39
dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
87
dbms/src/AggregateFunctions/AggregateFunctionBitwise.h
Normal file
87
dbms/src/AggregateFunctions/AggregateFunctionBitwise.h
Normal 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);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -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>
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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
|
@ -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;
|
Loading…
Reference in New Issue
Block a user