fix groupUniqArray

This commit is contained in:
Alexander Tokmakov 2021-07-26 17:57:49 +03:00
parent 5c9f69ea17
commit 304c367912
4 changed files with 24 additions and 17 deletions

View File

@ -25,8 +25,8 @@ template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>
{
public:
explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>(argument_type, max_elems_) {}
explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>(argument_type, parameters_, max_elems_) {}
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDate>()); }
};
@ -34,8 +34,8 @@ template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>
{
public:
explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>(argument_type, max_elems_) {}
explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>(argument_type, parameters_, max_elems_) {}
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>()); }
};
@ -102,9 +102,9 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!limit_size)
return createAggregateFunctionGroupUniqArrayImpl<std::false_type>(name, argument_types[0]);
return createAggregateFunctionGroupUniqArrayImpl<std::false_type>(name, argument_types[0], parameters);
else
return createAggregateFunctionGroupUniqArrayImpl<std::true_type>(name, argument_types[0], max_elems);
return createAggregateFunctionGroupUniqArrayImpl<std::true_type>(name, argument_types[0], parameters, max_elems);
}
}

View File

@ -48,9 +48,9 @@ private:
using State = AggregateFunctionGroupUniqArrayData<T>;
public:
AggregateFunctionGroupUniqArray(const DataTypePtr & argument_type, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
AggregateFunctionGroupUniqArray(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayData<T>,
AggregateFunctionGroupUniqArray<T, Tlimit_num_elem>>({argument_type}, {}),
AggregateFunctionGroupUniqArray<T, Tlimit_num_elem>>({argument_type}, parameters_),
max_elems(max_elems_) {}
String getName() const override { return "groupUniqArray"; }
@ -152,8 +152,8 @@ class AggregateFunctionGroupUniqArrayGeneric
using State = AggregateFunctionGroupUniqArrayGenericData;
public:
AggregateFunctionGroupUniqArrayGeneric(const DataTypePtr & input_data_type_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData, AggregateFunctionGroupUniqArrayGeneric<is_plain_column, Tlimit_num_elem>>({input_data_type_}, {})
AggregateFunctionGroupUniqArrayGeneric(const DataTypePtr & input_data_type_, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData, AggregateFunctionGroupUniqArrayGeneric<is_plain_column, Tlimit_num_elem>>({input_data_type_}, parameters_)
, input_data_type(this->argument_types[0])
, max_elems(max_elems_) {}

View File

@ -1,3 +1,6 @@
5 5
5 5
5 5
5 5
5 5
5 5

View File

@ -4,16 +4,20 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
declare -a functions=("groupArraySample" "groupUniqArray")
declare -a engines=("Memory" "MergeTree order by n" "Log")
for engine in "${engines[@]}"
for func in "${functions[@]}"
do
$CLICKHOUSE_CLIENT -q "drop table if exists t";
$CLICKHOUSE_CLIENT -q "create table t (n UInt8, a1 AggregateFunction(groupArraySample(1), UInt8)) engine=$engine"
$CLICKHOUSE_CLIENT -q "insert into t select number % 5 as n, groupArraySampleState(1)(toUInt8(number)) from numbers(10) group by n"
for engine in "${engines[@]}"
do
$CLICKHOUSE_CLIENT -q "drop table if exists t";
$CLICKHOUSE_CLIENT -q "create table t (n UInt8, a1 AggregateFunction($func(1), UInt8)) engine=$engine"
$CLICKHOUSE_CLIENT -q "insert into t select number % 5 as n, ${func}State(1)(toUInt8(number)) from numbers(10) group by n"
$CLICKHOUSE_CLIENT -q "select * from t format TSV" | $CLICKHOUSE_CLIENT -q "insert into t format TSV"
$CLICKHOUSE_CLIENT -q "select countDistinct(n), countDistinct(a1) from t"
$CLICKHOUSE_CLIENT -q "select * from t format TSV" | $CLICKHOUSE_CLIENT -q "insert into t format TSV"
$CLICKHOUSE_CLIENT -q "select countDistinct(n), countDistinct(a1) from t"
$CLICKHOUSE_CLIENT -q "drop table t";
$CLICKHOUSE_CLIENT -q "drop table t";
done
done