mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Specialized implementation of groupArray() for Date and DateTime. [#CLICKHOUSE-3213]
This commit is contained in:
parent
b7f8ac1f2d
commit
04dcec32b4
@ -8,6 +8,28 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
template <template <typename, typename> class AggregateFunctionTemplate, class Data, typename ... TArgs>
|
||||
static IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type, TArgs && ... args)
|
||||
{
|
||||
if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionTemplate<UInt16, Data>(std::forward<TArgs>(args)...);
|
||||
else if (typeid_cast<const DataTypeDateTime *>(&argument_type)) return new AggregateFunctionTemplate<UInt32, Data>(std::forward<TArgs>(args)...);
|
||||
else return createWithNumericType<AggregateFunctionTemplate, Data, TArgs...>(argument_type, std::forward<TArgs>(args)...);
|
||||
}
|
||||
|
||||
|
||||
template <typename TLimit_size, typename ... TArgs>
|
||||
inline AggregateFunctionPtr createAggregateFunctionGroupArrayImpl(const DataTypePtr & argument_type, TArgs ... args)
|
||||
{
|
||||
if (auto res = createWithNumericOrTimeType<GroupArrayNumericImpl, TLimit_size>(*argument_type, argument_type, std::forward<TArgs>(args)...))
|
||||
return AggregateFunctionPtr(res);
|
||||
|
||||
if (typeid_cast<const DataTypeString *>(argument_type.get()))
|
||||
return std::make_shared<GroupArrayGeneralListImpl<NodeString, TLimit_size::value>>(std::forward<TArgs>(args)...);
|
||||
|
||||
return std::make_shared<GroupArrayGeneralListImpl<NodeGeneral, TLimit_size::value>>(std::forward<TArgs>(args)...);
|
||||
};
|
||||
|
||||
|
||||
static AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
@ -15,7 +37,7 @@ static AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
bool limit_size = false;
|
||||
UInt64 max_elems = 0;
|
||||
UInt64 max_elems = std::numeric_limits<UInt64>::max();
|
||||
|
||||
if (parameters.empty())
|
||||
{
|
||||
@ -39,23 +61,9 @@ static AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!limit_size)
|
||||
{
|
||||
if (auto res = createWithNumericType<GroupArrayNumericImpl, std::false_type>(*argument_types[0]))
|
||||
return AggregateFunctionPtr(res);
|
||||
else if (typeid_cast<const DataTypeString *>(argument_types[0].get()))
|
||||
return std::make_shared<GroupArrayGeneralListImpl<NodeString, false>>();
|
||||
else
|
||||
return std::make_shared<GroupArrayGeneralListImpl<NodeGeneral, false>>();
|
||||
}
|
||||
return createAggregateFunctionGroupArrayImpl<std::false_type>(argument_types[0]);
|
||||
else
|
||||
{
|
||||
if (auto res = createWithNumericType<GroupArrayNumericImpl, std::true_type>(*argument_types[0], max_elems))
|
||||
return AggregateFunctionPtr(res);
|
||||
else if (typeid_cast<const DataTypeString *>(argument_types[0].get()))
|
||||
return std::make_shared<GroupArrayGeneralListImpl<NodeString, true>>(max_elems);
|
||||
else
|
||||
return std::make_shared<GroupArrayGeneralListImpl<NodeGeneral, true>>(max_elems);
|
||||
}
|
||||
return createAggregateFunctionGroupArrayImpl<std::true_type>(argument_types[0], max_elems);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -51,16 +51,18 @@ class GroupArrayNumericImpl final
|
||||
: public IUnaryAggregateFunction<GroupArrayNumericData<T>, GroupArrayNumericImpl<T, Tlimit_num_elems>>
|
||||
{
|
||||
static constexpr bool limit_num_elems = Tlimit_num_elems::value;
|
||||
DataTypePtr data_type;
|
||||
UInt64 max_elems;
|
||||
|
||||
public:
|
||||
GroupArrayNumericImpl(UInt64 max_elems_ = std::numeric_limits<UInt64>::max()) : max_elems(max_elems_) {}
|
||||
explicit GroupArrayNumericImpl(const DataTypePtr & data_type_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
|
||||
: data_type(data_type_), max_elems(max_elems_) {}
|
||||
|
||||
String getName() const override { return "groupArray"; }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeNumber<T>>());
|
||||
return std::make_shared<DataTypeArray>(data_type);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) {}
|
||||
|
@ -54,7 +54,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
|
||||
}
|
||||
|
||||
template <template <typename, typename> class AggregateFunctionTemplate, class Data, typename ... TArgs>
|
||||
static IAggregateFunction * createWithNumericType(const IDataType & argument_type, TArgs ... args)
|
||||
static IAggregateFunction * createWithNumericType(const IDataType & argument_type, TArgs && ... args)
|
||||
{
|
||||
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data>(std::forward<TArgs>(args)...);
|
||||
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<UInt16, Data>(std::forward<TArgs>(args)...);
|
||||
|
@ -10,6 +10,9 @@
|
||||
9 100
|
||||
|
||||
0 1000000
|
||||
|
||||
21
|
||||
41
|
||||
1000000 500000500000 1000000
|
||||
1000000 500000500000 1000000
|
||||
1000000 500000500000 1000000 500000500000
|
||||
|
@ -1,5 +1,9 @@
|
||||
SELECT intDiv(number, 100) AS k, length(groupArray(number)) FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
|
||||
SELECT '';
|
||||
SELECT length(toString(groupArrayState(toDate(number)))) FROM (SELECT * FROM system.numbers LIMIT 10);
|
||||
SELECT length(toString(groupArrayState(toDateTime(number)))) FROM (SELECT * FROM system.numbers LIMIT 10);
|
||||
|
||||
DROP TABLE IF EXISTS test.numbers_mt;
|
||||
CREATE TABLE test.numbers_mt (number UInt64) ENGINE = Log;
|
||||
INSERT INTO test.numbers_mt SELECT * FROM system.numbers LIMIT 1, 1000000;
|
||||
|
Loading…
Reference in New Issue
Block a user