mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 04:52:10 +00:00
fixup! Add groupSortedArray() function
This commit is contained in:
parent
7f553d55ae
commit
9c38b1a031
@ -42,73 +42,35 @@ namespace
|
|||||||
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<T>()); }
|
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<T>()); }
|
||||||
};
|
};
|
||||||
|
|
||||||
template <bool expr_sorted, typename TColumnB, bool is_plain_b>
|
template <bool expr_sorted = false, typename TColumnB = UInt64, bool is_plain_b = false>
|
||||||
static IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, const Array & params)
|
AggregateFunctionPtr
|
||||||
|
createAggregateFunctionGroupSortedArrayTyped(const DataTypes & argument_types, const Array & params, UInt64 threshold)
|
||||||
{
|
{
|
||||||
if (argument_types.empty())
|
#define DISPATCH(A, CLS, B) \
|
||||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Got empty arguments list");
|
if (which.idx == TypeIndex::A) \
|
||||||
|
return AggregateFunctionPtr(new CLS<B, expr_sorted, TColumnB, is_plain_b>(threshold, argument_types, params));
|
||||||
|
#define DISPATCH_NUMERIC(A) DISPATCH(A, AggregateFunctionGroupSortedArrayNumeric, A)
|
||||||
WhichDataType which(argument_types[0]);
|
WhichDataType which(argument_types[0]);
|
||||||
if (which.idx == TypeIndex::Date)
|
FOR_NUMERIC_TYPES(DISPATCH_NUMERIC)
|
||||||
return new AggregateFunctionGroupSortedArrayFieldType<DataTypeDate, expr_sorted, TColumnB, is_plain_b>(
|
DISPATCH(Enum8, AggregateFunctionGroupSortedArrayNumeric, Int8)
|
||||||
threshold, argument_types, params);
|
DISPATCH(Enum16, AggregateFunctionGroupSortedArrayNumeric, Int16)
|
||||||
if (which.idx == TypeIndex::DateTime)
|
DISPATCH(Date, AggregateFunctionGroupSortedArrayFieldType, DataTypeDate)
|
||||||
return new AggregateFunctionGroupSortedArrayFieldType<DataTypeDateTime, expr_sorted, TColumnB, is_plain_b>(
|
DISPATCH(DateTime, AggregateFunctionGroupSortedArrayFieldType, DataTypeDateTime)
|
||||||
threshold, argument_types, params);
|
#undef DISPATCH
|
||||||
|
#undef DISPATCH_NUMERIC
|
||||||
|
|
||||||
if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||||
{
|
{
|
||||||
return new AggregateFunctionGroupSortedArray<StringRef, true, expr_sorted, TColumnB, is_plain_b>(
|
return AggregateFunctionPtr(new AggregateFunctionGroupSortedArray<StringRef, true, expr_sorted, TColumnB, is_plain_b>(
|
||||||
threshold, argument_types, params);
|
threshold, argument_types, params));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
|
||||||
{
|
{
|
||||||
return new AggregateFunctionGroupSortedArray<StringRef, false, expr_sorted, TColumnB, is_plain_b>(
|
return AggregateFunctionPtr(new AggregateFunctionGroupSortedArray<StringRef, false, expr_sorted, TColumnB, is_plain_b>(
|
||||||
threshold, argument_types, params);
|
threshold, argument_types, params));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
template <
|
|
||||||
template <typename, bool, typename, bool>
|
|
||||||
class AggregateFunctionTemplate,
|
|
||||||
bool bool_param,
|
|
||||||
typename TColumnB,
|
|
||||||
bool is_plain_b,
|
|
||||||
typename... TArgs>
|
|
||||||
static IAggregateFunction * createWithNumericType2(const IDataType & argument_type, TArgs &&... args)
|
|
||||||
{
|
|
||||||
WhichDataType which(argument_type);
|
|
||||||
#define DISPATCH(TYPE) \
|
|
||||||
if (which.idx == TypeIndex::TYPE) \
|
|
||||||
return new AggregateFunctionTemplate<TYPE, bool_param, TColumnB, is_plain_b>(std::forward<TArgs>(args)...);
|
|
||||||
FOR_NUMERIC_TYPES(DISPATCH)
|
|
||||||
#undef DISPATCH
|
|
||||||
if (which.idx == TypeIndex::Enum8)
|
|
||||||
return new AggregateFunctionTemplate<Int8, bool_param, TColumnB, is_plain_b>(std::forward<TArgs>(args)...);
|
|
||||||
if (which.idx == TypeIndex::Enum16)
|
|
||||||
return new AggregateFunctionTemplate<Int16, bool_param, TColumnB, is_plain_b>(std::forward<TArgs>(args)...);
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
template <bool expr_sorted = false, typename TColumnB = UInt64, bool is_plain_b = false>
|
|
||||||
AggregateFunctionPtr createAggregateFunctionGroupSortedArrayTyped(
|
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, UInt64 threshold)
|
|
||||||
{
|
|
||||||
AggregateFunctionPtr res(createWithNumericType2<AggregateFunctionGroupSortedArrayNumeric, expr_sorted, TColumnB, is_plain_b>(
|
|
||||||
*argument_types[0], threshold, argument_types, params));
|
|
||||||
|
|
||||||
if (!res)
|
|
||||||
res = AggregateFunctionPtr(createWithExtraTypes<expr_sorted, TColumnB, is_plain_b>(argument_types, threshold, params));
|
|
||||||
|
|
||||||
if (!res)
|
|
||||||
throw Exception(
|
|
||||||
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
|
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionGroupSortedArray(
|
AggregateFunctionPtr createAggregateFunctionGroupSortedArray(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||||
@ -136,26 +98,32 @@ namespace
|
|||||||
|
|
||||||
if (argument_types.size() == 2)
|
if (argument_types.size() == 2)
|
||||||
{
|
{
|
||||||
if (isUnsignedInteger(argument_types[1]))
|
if (isNumber(argument_types[1]))
|
||||||
{
|
{
|
||||||
return createAggregateFunctionGroupSortedArrayTyped<true, UInt64>(name, argument_types, params, threshold);
|
#define DISPATCH2(A, B) \
|
||||||
}
|
if (which.idx == TypeIndex::A) \
|
||||||
else if (isInteger(argument_types[1]))
|
return createAggregateFunctionGroupSortedArrayTyped<true, B>(argument_types, params, threshold);
|
||||||
{
|
#define DISPATCH(A) DISPATCH2(A, A)
|
||||||
return createAggregateFunctionGroupSortedArrayTyped<true, Int64>(name, argument_types, params, threshold);
|
WhichDataType which(argument_types[1]);
|
||||||
|
FOR_NUMERIC_TYPES(DISPATCH)
|
||||||
|
DISPATCH2(Enum8, Int8)
|
||||||
|
DISPATCH2(Enum16, Int16)
|
||||||
|
#undef DISPATCH
|
||||||
|
#undef DISPATCH2
|
||||||
|
throw Exception("Invalid parameter type.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
}
|
}
|
||||||
else if (argument_types[1]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
else if (argument_types[1]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||||
{
|
{
|
||||||
return createAggregateFunctionGroupSortedArrayTyped<true, StringRef, true>(name, argument_types, params, threshold);
|
return createAggregateFunctionGroupSortedArrayTyped<true, StringRef, true>(argument_types, params, threshold);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
return createAggregateFunctionGroupSortedArrayTyped<true, StringRef, false>(name, argument_types, params, threshold);
|
return createAggregateFunctionGroupSortedArrayTyped<true, StringRef, false>(argument_types, params, threshold);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (argument_types.size() == 1)
|
else if (argument_types.size() == 1)
|
||||||
{
|
{
|
||||||
return createAggregateFunctionGroupSortedArrayTyped<>(name, argument_types, params, threshold);
|
return createAggregateFunctionGroupSortedArrayTyped<>(argument_types, params, threshold);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
Loading…
Reference in New Issue
Block a user