mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
fix groupUniqArray
This commit is contained in:
parent
5c9f69ea17
commit
304c367912
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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_) {}
|
||||
|
||||
|
@ -1,3 +1,6 @@
|
||||
5 5
|
||||
5 5
|
||||
5 5
|
||||
5 5
|
||||
5 5
|
||||
5 5
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user