mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
fix parameters with combinators
This commit is contained in:
parent
4862e9f80d
commit
5c9f69ea17
@ -43,9 +43,9 @@ public:
|
|||||||
const AggregateFunctionPtr & nested_function,
|
const AggregateFunctionPtr & nested_function,
|
||||||
const AggregateFunctionProperties &,
|
const AggregateFunctionProperties &,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array &) const override
|
const Array & params) const override
|
||||||
{
|
{
|
||||||
return std::make_shared<AggregateFunctionArray>(nested_function, arguments);
|
return std::make_shared<AggregateFunctionArray>(nested_function, arguments, params);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -29,10 +29,11 @@ private:
|
|||||||
size_t num_arguments;
|
size_t num_arguments;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
AggregateFunctionArray(AggregateFunctionPtr nested_, const DataTypes & arguments)
|
AggregateFunctionArray(AggregateFunctionPtr nested_, const DataTypes & arguments, const Array & params_)
|
||||||
: IAggregateFunctionHelper<AggregateFunctionArray>(arguments, {})
|
: IAggregateFunctionHelper<AggregateFunctionArray>(arguments, params_)
|
||||||
, nested_func(nested_), num_arguments(arguments.size())
|
, nested_func(nested_), num_arguments(arguments.size())
|
||||||
{
|
{
|
||||||
|
assert(parameters == nested_func->getParameters());
|
||||||
for (const auto & type : arguments)
|
for (const auto & type : arguments)
|
||||||
if (!isArray(type))
|
if (!isArray(type))
|
||||||
throw Exception("All arguments for aggregate function " + getName() + " must be arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("All arguments for aggregate function " + getName() + " must be arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
@ -34,14 +34,14 @@ public:
|
|||||||
const AggregateFunctionPtr & nested_function,
|
const AggregateFunctionPtr & nested_function,
|
||||||
const AggregateFunctionProperties &,
|
const AggregateFunctionProperties &,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array &) const override
|
const Array & params) const override
|
||||||
{
|
{
|
||||||
AggregateFunctionPtr res;
|
AggregateFunctionPtr res;
|
||||||
if (arguments.size() == 1)
|
if (arguments.size() == 1)
|
||||||
{
|
{
|
||||||
res.reset(createWithNumericType<
|
res.reset(createWithNumericType<
|
||||||
AggregateFunctionDistinct,
|
AggregateFunctionDistinct,
|
||||||
AggregateFunctionDistinctSingleNumericData>(*arguments[0], nested_function, arguments));
|
AggregateFunctionDistinctSingleNumericData>(*arguments[0], nested_function, arguments, params));
|
||||||
|
|
||||||
if (res)
|
if (res)
|
||||||
return res;
|
return res;
|
||||||
@ -49,14 +49,14 @@ public:
|
|||||||
if (arguments[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
if (arguments[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||||
return std::make_shared<
|
return std::make_shared<
|
||||||
AggregateFunctionDistinct<
|
AggregateFunctionDistinct<
|
||||||
AggregateFunctionDistinctSingleGenericData<true>>>(nested_function, arguments);
|
AggregateFunctionDistinctSingleGenericData<true>>>(nested_function, arguments, params);
|
||||||
else
|
else
|
||||||
return std::make_shared<
|
return std::make_shared<
|
||||||
AggregateFunctionDistinct<
|
AggregateFunctionDistinct<
|
||||||
AggregateFunctionDistinctSingleGenericData<false>>>(nested_function, arguments);
|
AggregateFunctionDistinctSingleGenericData<false>>>(nested_function, arguments, params);
|
||||||
}
|
}
|
||||||
|
|
||||||
return std::make_shared<AggregateFunctionDistinct<AggregateFunctionDistinctMultipleGenericData>>(nested_function, arguments);
|
return std::make_shared<AggregateFunctionDistinct<AggregateFunctionDistinctMultipleGenericData>>(nested_function, arguments, params);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -167,8 +167,8 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
public:
|
public:
|
||||||
AggregateFunctionDistinct(AggregateFunctionPtr nested_func_, const DataTypes & arguments)
|
AggregateFunctionDistinct(AggregateFunctionPtr nested_func_, const DataTypes & arguments, const Array & params_)
|
||||||
: IAggregateFunctionDataHelper<Data, AggregateFunctionDistinct>(arguments, nested_func_->getParameters())
|
: IAggregateFunctionDataHelper<Data, AggregateFunctionDistinct>(arguments, params_)
|
||||||
, nested_func(nested_func_)
|
, nested_func(nested_func_)
|
||||||
, arguments_num(arguments.size()) {}
|
, arguments_num(arguments.size()) {}
|
||||||
|
|
||||||
|
@ -38,9 +38,9 @@ public:
|
|||||||
const AggregateFunctionPtr & nested_function,
|
const AggregateFunctionPtr & nested_function,
|
||||||
const AggregateFunctionProperties &,
|
const AggregateFunctionProperties &,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array &) const override
|
const Array & params) const override
|
||||||
{
|
{
|
||||||
return std::make_shared<AggregateFunctionForEach>(nested_function, arguments);
|
return std::make_shared<AggregateFunctionForEach>(nested_function, arguments, params);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -105,8 +105,8 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
public:
|
public:
|
||||||
AggregateFunctionForEach(AggregateFunctionPtr nested_, const DataTypes & arguments)
|
AggregateFunctionForEach(AggregateFunctionPtr nested_, const DataTypes & arguments, const Array & params_)
|
||||||
: IAggregateFunctionDataHelper<AggregateFunctionForEachData, AggregateFunctionForEach>(arguments, {})
|
: IAggregateFunctionDataHelper<AggregateFunctionForEachData, AggregateFunctionForEach>(arguments, params_)
|
||||||
, nested_func(nested_), num_arguments(arguments.size())
|
, nested_func(nested_), num_arguments(arguments.size())
|
||||||
{
|
{
|
||||||
nested_size_of_data = nested_func->sizeOfData();
|
nested_size_of_data = nested_func->sizeOfData();
|
||||||
|
@ -35,9 +35,9 @@ public:
|
|||||||
const AggregateFunctionPtr & nested_function,
|
const AggregateFunctionPtr & nested_function,
|
||||||
const AggregateFunctionProperties &,
|
const AggregateFunctionProperties &,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array &) const override
|
const Array & params) const override
|
||||||
{
|
{
|
||||||
return std::make_shared<AggregateFunctionIf>(nested_function, arguments);
|
return std::make_shared<AggregateFunctionIf>(nested_function, arguments, params);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -37,8 +37,8 @@ private:
|
|||||||
size_t num_arguments;
|
size_t num_arguments;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
AggregateFunctionIf(AggregateFunctionPtr nested, const DataTypes & types)
|
AggregateFunctionIf(AggregateFunctionPtr nested, const DataTypes & types, const Array & params_)
|
||||||
: IAggregateFunctionHelper<AggregateFunctionIf>(types, nested->getParameters())
|
: IAggregateFunctionHelper<AggregateFunctionIf>(types, params_)
|
||||||
, nested_func(nested), num_arguments(types.size())
|
, nested_func(nested), num_arguments(types.size())
|
||||||
{
|
{
|
||||||
if (num_arguments == 0)
|
if (num_arguments == 0)
|
||||||
|
@ -39,7 +39,7 @@ public:
|
|||||||
const AggregateFunctionPtr & nested_function,
|
const AggregateFunctionPtr & nested_function,
|
||||||
const AggregateFunctionProperties &,
|
const AggregateFunctionProperties &,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array &) const override
|
const Array & params) const override
|
||||||
{
|
{
|
||||||
const DataTypePtr & argument = arguments[0];
|
const DataTypePtr & argument = arguments[0];
|
||||||
|
|
||||||
@ -53,7 +53,7 @@ public:
|
|||||||
+ ", because it corresponds to different aggregate function: " + function->getFunctionName() + " instead of " + nested_function->getName(),
|
+ ", because it corresponds to different aggregate function: " + function->getFunctionName() + " instead of " + nested_function->getName(),
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
return std::make_shared<AggregateFunctionMerge>(nested_function, argument);
|
return std::make_shared<AggregateFunctionMerge>(nested_function, argument, params);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -29,8 +29,8 @@ private:
|
|||||||
AggregateFunctionPtr nested_func;
|
AggregateFunctionPtr nested_func;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
AggregateFunctionMerge(const AggregateFunctionPtr & nested_, const DataTypePtr & argument)
|
AggregateFunctionMerge(const AggregateFunctionPtr & nested_, const DataTypePtr & argument, const Array & params_)
|
||||||
: IAggregateFunctionHelper<AggregateFunctionMerge>({argument}, nested_->getParameters())
|
: IAggregateFunctionHelper<AggregateFunctionMerge>({argument}, params_)
|
||||||
, nested_func(nested_)
|
, nested_func(nested_)
|
||||||
{
|
{
|
||||||
const DataTypeAggregateFunction * data_type = typeid_cast<const DataTypeAggregateFunction *>(argument.get());
|
const DataTypeAggregateFunction * data_type = typeid_cast<const DataTypeAggregateFunction *>(argument.get());
|
||||||
|
@ -0,0 +1,20 @@
|
|||||||
|
AggregateFunction(topKArray(10), Array(String))
|
||||||
|
AggregateFunction(topKDistinct(10), String)
|
||||||
|
AggregateFunction(topKForEach(10), Array(String))
|
||||||
|
AggregateFunction(topKIf(10), String, UInt8)
|
||||||
|
AggregateFunction(topK(10), String)
|
||||||
|
AggregateFunction(topKOrNull(10), String)
|
||||||
|
AggregateFunction(topKOrDefault(10), String)
|
||||||
|
AggregateFunction(topKResample(10, 1, 2, 42), String, UInt64)
|
||||||
|
AggregateFunction(topK(10), String)
|
||||||
|
AggregateFunction(topKArrayResampleOrDefaultIf(10, 1, 2, 42), Array(String), UInt64, UInt8)
|
||||||
|
10
|
||||||
|
10
|
||||||
|
[10]
|
||||||
|
11
|
||||||
|
10
|
||||||
|
10
|
||||||
|
10
|
||||||
|
[1]
|
||||||
|
10
|
||||||
|
[1]
|
@ -0,0 +1,43 @@
|
|||||||
|
SELECT toTypeName(topKArrayState(10)([toString(number)])) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKDistinctState(10)(toString(number))) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKForEachState(10)([toString(number)])) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKIfState(10)(toString(number), number % 2)) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKMergeState(10)(state)) FROM (SELECT topKState(10)(toString(number)) as state FROM numbers(100));
|
||||||
|
SELECT toTypeName(topKOrNullState(10)(toString(number))) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKOrDefaultState(10)(toString(number))) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKResampleState(10, 1, 2, 42)(toString(number), number)) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKState(10)(toString(number))) FROM numbers(100);
|
||||||
|
SELECT toTypeName(topKArrayResampleOrDefaultIfState(10, 1, 2, 42)([toString(number)], number, number % 2)) FROM numbers(100);
|
||||||
|
|
||||||
|
CREATE TEMPORARY TABLE t0 AS SELECT quantileArrayState(0.10)([number]) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t1 AS SELECT quantileDistinctState(0.10)(number) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t2 AS SELECT quantileForEachState(0.10)([number]) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t3 AS SELECT quantileIfState(0.10)(number, number % 2) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t4 AS SELECT quantileMergeState(0.10)(state) FROM (SELECT quantileState(0.10)(number) as state FROM numbers(100));
|
||||||
|
CREATE TEMPORARY TABLE t5 AS SELECT quantileOrNullState(0.10)(number) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t6 AS SELECT quantileOrDefaultState(0.10)(number) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t7 AS SELECT quantileResampleState(0.10, 1, 2, 42)(number, number) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t8 AS SELECT quantileState(0.10)(number) FROM numbers(100);
|
||||||
|
CREATE TEMPORARY TABLE t9 AS SELECT quantileArrayResampleOrDefaultIfState(0.10, 1, 2, 42)([number], number, number % 2) FROM numbers(100);
|
||||||
|
|
||||||
|
INSERT INTO t0 SELECT quantileArrayState(0.10)([number]) FROM numbers(100);
|
||||||
|
INSERT INTO t1 SELECT quantileDistinctState(0.10)(number) FROM numbers(100);
|
||||||
|
INSERT INTO t2 SELECT quantileForEachState(0.10)([number]) FROM numbers(100);
|
||||||
|
INSERT INTO t3 SELECT quantileIfState(0.10)(number, number % 2) FROM numbers(100);
|
||||||
|
INSERT INTO t4 SELECT quantileMergeState(0.10)(state) FROM (SELECT quantileState(0.10)(number) as state FROM numbers(100));
|
||||||
|
INSERT INTO t5 SELECT quantileOrNullState(0.10)(number) FROM numbers(100);
|
||||||
|
INSERT INTO t6 SELECT quantileOrDefaultState(0.10)(number) FROM numbers(100);
|
||||||
|
INSERT INTO t7 SELECT quantileResampleState(0.10, 1, 2, 42)(number, number) FROM numbers(100);
|
||||||
|
INSERT INTO t8 SELECT quantileState(0.10)(number) FROM numbers(100);
|
||||||
|
INSERT INTO t9 SELECT quantileArrayResampleOrDefaultIfState(0.10, 1, 2, 42)([number], number, number % 2) FROM numbers(100);
|
||||||
|
|
||||||
|
SELECT round(quantileArrayMerge(0.10)((*,).1)) FROM t0;
|
||||||
|
SELECT round(quantileDistinctMerge(0.10)((*,).1)) FROM t1;
|
||||||
|
SELECT arrayMap(x -> round(x), quantileForEachMerge(0.10)((*,).1)) FROM t2;
|
||||||
|
SELECT round(quantileIfMerge(0.10)((*,).1)) FROM t3;
|
||||||
|
SELECT round(quantileMerge(0.10)((*,).1)) FROM t4;
|
||||||
|
SELECT round(quantileOrNullMerge(0.10)((*,).1)) FROM t5;
|
||||||
|
SELECT round(quantileOrDefaultMerge(0.10)((*,).1)) FROM t6;
|
||||||
|
SELECT arrayMap(x -> round(x), quantileResampleMerge(0.10, 1, 2, 42)((*,).1)) FROM t7;
|
||||||
|
SELECT round(quantileMerge(0.10)((*,).1)) FROM t8;
|
||||||
|
SELECT arrayMap(x -> round(x), quantileArrayResampleOrDefaultIfMerge(0.10, 1, 2, 42)((*,).1)) FROM t9;
|
Loading…
Reference in New Issue
Block a user