Merge pull request #25015 from ClickHouse/fix-wrong-code-in-aggregate-functions

Remove a chunk of wrong code and look what will happen
This commit is contained in:
alexey-milovidov 2021-06-11 00:36:35 +03:00 committed by GitHub
commit 7616216f10
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
70 changed files with 209 additions and 219 deletions

View File

@ -1,7 +1,7 @@
#include <AggregateFunctions/AggregateFunctionArray.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionBitwise.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionBoundingRatio.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -3,7 +3,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,11 +1,9 @@
#include <Common/StringUtils/StringUtils.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -100,7 +100,7 @@ public:
}
/// Reset the state to specified value. This function is not the part of common interface.
void set(AggregateDataPtr __restrict place, UInt64 new_count)
void set(AggregateDataPtr __restrict place, UInt64 new_count) const
{
data(place).count = new_count;
}

View File

@ -2,7 +2,7 @@
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionEntropy.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -17,7 +16,8 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionEntropy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionEntropy(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
if (argument_types.empty())

View File

@ -17,10 +17,10 @@
#include <Common/CurrentThread.h>
#include <Poco/String.h>
#include "registerAggregateFunctions.h"
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Settings;
@ -95,7 +95,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
// nullability themselves. Another special case is functions from Nothing
// that are rewritten to AggregateFunctionNothing, in this case
// nested_function is nullptr.
if (nested_function && nested_function->asWindowFunction())
if (nested_function && nested_function->isOnlyWindowFunction())
{
return nested_function;
}

View File

@ -1,12 +1,10 @@
#include <AggregateFunctions/AggregateFunctionForEach.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -4,7 +4,6 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -50,7 +49,8 @@ inline AggregateFunctionPtr createAggregateFunctionGroupArrayImpl(const DataType
}
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionGroupArray(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -17,7 +16,8 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);

View File

@ -5,7 +5,6 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -58,7 +57,8 @@ inline AggregateFunctionPtr createAggregateFunctionMovingImpl(const std::string
}
template <template <typename, typename> class Function>
AggregateFunctionPtr createAggregateFunctionMoving(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionMoving(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -2,11 +2,11 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include "registerAggregateFunctions.h"
// TODO include this last because of a broken roaring header. See the comment inside.
#include <AggregateFunctions/AggregateFunctionGroupBitmap.h>
namespace DB
{
struct Settings;
@ -18,9 +18,8 @@ namespace ErrorCodes
namespace
{
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data, typename... TArgs>
static IAggregateFunction * createWithIntegerType(const IDataType & argument_type, TArgs && ... args)
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> typename Data, typename... TArgs>
static IAggregateFunction * createWithIntegerType(const IDataType & argument_type, TArgs &&... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::UInt8) return new AggregateFunctionTemplate<UInt8, Data<UInt8>>(std::forward<TArgs>(args)...);
@ -34,8 +33,9 @@ namespace
return nullptr;
}
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionBitmap(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename> typename Data>
AggregateFunctionPtr createAggregateFunctionBitmap(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
@ -57,12 +57,13 @@ namespace
}
// Additional aggregate functions to manipulate bitmaps.
template <template <typename, typename> class AggregateFunctionTemplate>
AggregateFunctionPtr
createAggregateFunctionBitmapL2(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename, typename> typename AggregateFunctionTemplate>
AggregateFunctionPtr createAggregateFunctionBitmapL2(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
DataTypePtr argument_type_ptr = argument_types[0];
WhichDataType which(*argument_type_ptr);
if (which.idx != TypeIndex::AggregateFunction)
@ -70,11 +71,15 @@ namespace
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// groupBitmap needs to know about the data type that was used to create bitmaps.
/// We need to look inside the type of its argument to obtain it.
const DataTypeAggregateFunction & datatype_aggfunc = dynamic_cast<const DataTypeAggregateFunction &>(*argument_type_ptr);
AggregateFunctionPtr aggfunc = datatype_aggfunc.getFunction();
argument_type_ptr = aggfunc->getArgumentTypes()[0];
DataTypePtr nested_argument_type_ptr = aggfunc->getArgumentTypes()[0];
AggregateFunctionPtr res(createWithIntegerType<AggregateFunctionTemplate, AggregateFunctionGroupBitmapData>(
*argument_type_ptr, argument_type_ptr));
*nested_argument_type_ptr, argument_type_ptr));
if (!res)
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,

View File

@ -9,15 +9,19 @@
// TODO include this last because of a broken roaring header. See the comment inside.
#include <AggregateFunctions/AggregateFunctionGroupBitmapData.h>
namespace DB
{
struct Settings;
/// Counts bitmap operation on numbers.
template <typename T, typename Data>
class AggregateFunctionBitmap final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>
{
public:
AggregateFunctionBitmap(const DataTypePtr & type) : IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>({type}, {}) { }
AggregateFunctionBitmap(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>({type}, {})
{
}
String getName() const override { return Data::name(); }
@ -46,6 +50,7 @@ public:
};
/// This aggregate function takes the states of AggregateFunctionBitmap as its argument.
template <typename T, typename Data, typename Policy>
class AggregateFunctionBitmapL2 final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmapL2<T, Data, Policy>>
{
@ -61,6 +66,11 @@ public:
bool allocatesMemoryInArena() const override { return false; }
DataTypePtr getStateType() const override
{
return this->argument_types.at(0);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
Data & data_lhs = this->data(place);
@ -105,6 +115,7 @@ public:
}
};
template <typename Data>
class BitmapAndPolicy
{

View File

@ -1,9 +1,9 @@
#pragma once
#include <algorithm>
#include <boost/noncopyable.hpp>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <boost/noncopyable.hpp>
#include <Common/HashTable/SmallTable.h>
#include <Common/PODArray.h>
@ -14,9 +14,9 @@
#include <roaring.hh>
#include <roaring64map.hh>
namespace DB
{
struct Settings;
enum BitmapKind
{
@ -24,6 +24,7 @@ enum BitmapKind
Bitmap = 1
};
/**
* For a small number of values - an array of fixed size "on the stack".
* For large, roaring bitmap is allocated.

View File

@ -4,7 +4,6 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -73,7 +72,8 @@ inline AggregateFunctionPtr createAggregateFunctionGroupUniqArrayImpl(const std:
}
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -4,7 +4,7 @@
#include <AggregateFunctions/Helpers.h>
#include <Common/FieldVisitors.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -1,12 +1,10 @@
#include <AggregateFunctions/AggregateFunctionIf.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
#include "AggregateFunctionNull.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -11,7 +11,6 @@
#include "AggregateFunctionFactory.h"
#include "FactoryHelpers.h"
#include "Helpers.h"
#include "registerAggregateFunctions.h"
namespace DB
@ -25,13 +24,14 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
using FuncLinearRegression = AggregateFunctionMLMethod<LinearModelData, NameLinearRegression>;
using FuncLogisticRegression = AggregateFunctionMLMethod<LinearModelData, NameLogisticRegression>;
template <class Method>
AggregateFunctionPtr
createAggregateFunctionMLMethod(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <typename Method>
AggregateFunctionPtr createAggregateFunctionMLMethod(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
if (parameters.size() > 4)
throw Exception(

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionMannWhitney.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
#include <AggregateFunctions/Helpers.h>
@ -17,7 +16,8 @@ struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/HelpersMinMaxAny.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -11,12 +10,14 @@ struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
AggregateFunctionPtr createAggregateFunctionMax(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
AggregateFunctionPtr createAggregateFunctionArgMax(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types, parameters, settings));
}

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionMaxIntersections.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,12 +1,10 @@
#include <AggregateFunctions/AggregateFunctionMerge.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -50,6 +50,11 @@ public:
return nested_func->getReturnType();
}
DataTypePtr getStateType() const override
{
return nested_func->getStateType();
}
void create(AggregateDataPtr __restrict place) const override
{
nested_func->create(place);

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/HelpersMinMaxAny.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -11,12 +10,14 @@ struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
AggregateFunctionPtr createAggregateFunctionMin(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
AggregateFunctionPtr createAggregateFunctionArgMin(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types, parameters, settings));
}

View File

@ -700,13 +700,11 @@ template <typename Data>
class AggregateFunctionsSingleValue final : public IAggregateFunctionDataHelper<Data, AggregateFunctionsSingleValue<Data>>
{
private:
DataTypePtr type;
SerializationPtr serialization;
public:
AggregateFunctionsSingleValue(const DataTypePtr & type_)
: IAggregateFunctionDataHelper<Data, AggregateFunctionsSingleValue<Data>>({type_}, {})
, type(this->argument_types[0])
AggregateFunctionsSingleValue(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionsSingleValue<Data>>({type}, {})
, serialization(type->getDefaultSerialization())
{
if (StringRef(Data::name()) == StringRef("min")
@ -722,7 +720,7 @@ public:
DataTypePtr getReturnType() const override
{
return type;
return this->argument_types.at(0);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override

View File

@ -4,12 +4,10 @@
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <AggregateFunctions/AggregateFunctionState.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -1,12 +1,11 @@
#include <AggregateFunctions/AggregateFunctionOrFill.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace
{

View File

@ -4,7 +4,7 @@
#include <AggregateFunctions/Helpers.h>
#include <Core/Field.h>
#include "registerAggregateFunctions.h"
namespace DB
{
@ -84,7 +84,8 @@ static constexpr bool supportBigInt()
}
template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);

View File

@ -1,13 +1,12 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionRankCorrelation.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
#include <AggregateFunctions/Helpers.h>
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int NOT_IMPLEMENTED;
}
namespace DB
@ -17,7 +16,8 @@ struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionRankCorrelation(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionRankCorrelation(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);

View File

@ -1,12 +1,10 @@
#include <AggregateFunctions/AggregateFunctionResample.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionRetention.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -6,7 +6,6 @@
#include <DataTypes/DataTypeDateTime.h>
#include <ext/range.h>
#include "registerAggregateFunctions.h"
namespace DB
{
@ -23,8 +22,9 @@ namespace ErrorCodes
namespace
{
template <template <typename, typename> class AggregateFunction, template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionSequenceBase(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
template <template <typename, typename> typename AggregateFunction, template <typename> typename Data>
AggregateFunctionPtr createAggregateFunctionSequenceBase(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (params.size() != 1)
throw Exception{"Aggregate function " + name + " requires exactly one parameter.",

View File

@ -2,17 +2,10 @@
#include <AggregateFunctions/AggregateFunctionMerge.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace
{
@ -44,36 +37,4 @@ void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorState>());
}
DataTypePtr AggregateFunctionState::getReturnType() const
{
auto ptr = std::make_shared<DataTypeAggregateFunction>(nested_func, arguments, params);
/// Special case: it is -MergeState combinator.
/// We must return AggregateFunction(agg, ...) instead of AggregateFunction(aggMerge, ...)
if (typeid_cast<const AggregateFunctionMerge *>(ptr->getFunction().get()))
{
if (arguments.size() != 1)
throw Exception("Combinator -MergeState expects only one argument", ErrorCodes::BAD_ARGUMENTS);
if (!typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get()))
throw Exception("Combinator -MergeState expects argument with AggregateFunction type", ErrorCodes::BAD_ARGUMENTS);
return arguments[0];
}
if (!arguments.empty())
{
DataTypePtr argument_type_ptr = arguments[0];
WhichDataType which(*argument_type_ptr);
if (which.idx == TypeIndex::AggregateFunction)
{
if (arguments.size() != 1)
throw Exception("Nested aggregation expects only one argument", ErrorCodes::BAD_ARGUMENTS);
return arguments[0];
}
}
return ptr;
}
}

View File

@ -33,7 +33,15 @@ public:
return nested_func->getName() + "State";
}
DataTypePtr getReturnType() const override;
DataTypePtr getReturnType() const override
{
return getStateType();
}
DataTypePtr getStateType() const override
{
return nested_func->getStateType();
}
void create(AggregateDataPtr __restrict place) const override
{

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/AggregateFunctionStatistics.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -17,8 +16,9 @@ namespace ErrorCodes
namespace
{
template <template <typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename> typename FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
@ -31,8 +31,9 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string &
return res;
}
template <template <typename, typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename, typename> typename FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertBinary(name, argument_types);

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/AggregateFunctionStatisticsSimple.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -17,8 +16,9 @@ namespace ErrorCodes
namespace
{
template <template <typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename> typename FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
@ -36,8 +36,9 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string &
return res;
}
template <template <typename, typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename, typename> typename FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertBinary(name, argument_types);

View File

@ -3,8 +3,6 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Moments.h>
#include "registerAggregateFunctions.h"
namespace ErrorCodes
{
@ -57,7 +55,8 @@ struct StudentTTestData : public TTestMoments<Float64>
}
};
AggregateFunctionPtr createAggregateFunctionStudentTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionStudentTTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionSum.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,7 +2,7 @@
#include <AggregateFunctions/AggregateFunctionSumCount.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -4,7 +4,6 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <Functions/FunctionHelpers.h>
#include <IO/WriteHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -11,12 +11,12 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -38,12 +38,6 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (argument_types.size() == 1)
{
const IDataType & argument_type = *argument_types[0];
@ -86,12 +80,6 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// We use exact hash function if the user wants it;
/// or if the arguments are not contiguous in memory, because only exact hash function have support for this case.
bool use_exact_hash_function = is_exact || !isAllArgumentsContiguousInMemory(argument_types);

View File

@ -13,13 +13,13 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
@ -105,11 +105,6 @@ namespace
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
switch (precision)
{

View File

@ -10,13 +10,13 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -47,11 +47,6 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types);

View File

@ -3,8 +3,6 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Moments.h>
#include "registerAggregateFunctions.h"
namespace ErrorCodes
{
@ -54,7 +52,8 @@ struct WelchTTestData : public TTestMoments<Float64>
}
};
AggregateFunctionPtr createAggregateFunctionWelchTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionWelchTTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);

View File

@ -2,6 +2,7 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
add_headers_and_sources(clickhouse_aggregate_functions .)
list(REMOVE_ITEM clickhouse_aggregate_functions_sources
IAggregateFunction.cpp
AggregateFunctionFactory.cpp
AggregateFunctionCombinatorFactory.cpp
AggregateFunctionCount.cpp

View File

@ -0,0 +1,13 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <DataTypes/DataTypeAggregateFunction.h>
namespace DB
{
DataTypePtr IAggregateFunction::getStateType() const
{
return std::make_shared<DataTypeAggregateFunction>(shared_from_this(), argument_types, parameters);
}
}

View File

@ -38,7 +38,7 @@ using AggregateDataPtr = char *;
using ConstAggregateDataPtr = const char *;
class IAggregateFunction;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
struct AggregateFunctionProperties;
/** Aggregate functions interface.
@ -49,7 +49,7 @@ struct AggregateFunctionProperties;
* (which can be created in some memory pool),
* and IAggregateFunction is the external interface for manipulating them.
*/
class IAggregateFunction
class IAggregateFunction : public std::enable_shared_from_this<IAggregateFunction>
{
public:
IAggregateFunction(const DataTypes & argument_types_, const Array & parameters_)
@ -61,6 +61,9 @@ public:
/// Get the result type.
virtual DataTypePtr getReturnType() const = 0;
/// Get the data type of internal state. By default it is AggregateFunction(name(params), argument_types...).
virtual DataTypePtr getStateType() const;
/// Get type which will be used for prediction result in case if function is an ML method.
virtual DataTypePtr getReturnTypeToPredict() const
{
@ -236,9 +239,7 @@ public:
// aggregate functions implement IWindowFunction interface and so on. This
// would be more logically correct, but more complex. We only have a handful
// of true window functions, so this hack-ish interface suffices.
virtual IWindowFunction * asWindowFunction() { return nullptr; }
virtual const IWindowFunction * asWindowFunction() const
{ return const_cast<IAggregateFunction *>(this)->asWindowFunction(); }
virtual bool isOnlyWindowFunction() const { return false; }
protected:
DataTypes argument_types;

View File

@ -61,6 +61,7 @@ SRCS(
AggregateFunctionUniqUpTo.cpp
AggregateFunctionWelchTTest.cpp
AggregateFunctionWindowFunnel.cpp
IAggregateFunction.cpp
UniqCombinedBiasData.cpp
UniqVariadicHash.cpp
parseAggregateFunctionParameters.cpp

View File

@ -111,6 +111,7 @@ list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp
list (APPEND dbms_headers Functions/IFunction.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h)
list (APPEND dbms_sources
AggregateFunctions/IAggregateFunction.cpp
AggregateFunctions/AggregateFunctionFactory.cpp
AggregateFunctions/AggregateFunctionCombinatorFactory.cpp
AggregateFunctions/AggregateFunctionState.cpp

View File

@ -167,7 +167,7 @@ MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAnd
MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn();
res->reserve(data.size());
auto * machine_learning_function = func.get();
const auto * machine_learning_function = func.get();
if (machine_learning_function)
{
if (data.size() == 1)
@ -485,7 +485,7 @@ Arena & ColumnAggregateFunction::createOrGetArena()
}
static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Arena & arena, IAggregateFunction * func)
static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Arena & arena, const IAggregateFunction * func)
{
data.push_back(arena.alignedAlloc(func->sizeOfData(), func->alignOfData()));
try

View File

@ -100,7 +100,7 @@ bool DataTypeAggregateFunction::equals(const IDataType & rhs) const
SerializationPtr DataTypeAggregateFunction::doGetDefaultSerialization() const
{
return std::make_shared<SerializationAggregateFunction>(function);
return std::make_shared<SerializationAggregateFunction>(function, getName());
}

View File

@ -20,19 +20,16 @@ namespace DB
void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
const String & s = get<const String &>(field);
writeVarUInt(s.size(), ostr);
writeString(s, ostr);
const AggregateFunctionStateData & state = get<const AggregateFunctionStateData &>(field);
writeBinary(state.data, ostr);
}
void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const
{
UInt64 size;
readVarUInt(size, istr);
field = String();
String & s = get<String &>(field);
s.resize(size);
istr.readStrict(s.data(), size);
field = AggregateFunctionStateData();
AggregateFunctionStateData & s = get<AggregateFunctionStateData &>(field);
readBinary(s.data, istr);
s.name = type_name;
}
void SerializationAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const

View File

@ -12,11 +12,13 @@ class SerializationAggregateFunction final : public ISerialization
{
private:
AggregateFunctionPtr function;
String type_name;
public:
static constexpr bool is_parametric = true;
SerializationAggregateFunction(const AggregateFunctionPtr & function_): function(function_) {}
SerializationAggregateFunction(const AggregateFunctionPtr & function_, String type_name_)
: function(function_), type_name(std::move(type_name_)) {}
/// NOTE These two functions for serializing single values are incompatible with the functions below.
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;

View File

@ -107,7 +107,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName
ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
IAggregateFunction & agg_func = *aggregate_function;
const IAggregateFunction & agg_func = *aggregate_function;
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
@ -178,9 +178,9 @@ ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & argume
});
{
auto * that = &agg_func;
const auto * that = &agg_func;
/// Unnest consecutive trailing -State combinators
while (auto * func = typeid_cast<AggregateFunctionState *>(that))
while (const auto * func = typeid_cast<const AggregateFunctionState *>(that))
that = func->getNestedFunction().get();
that->addBatchArray(input_rows_count, places.data(), 0, aggregate_arguments, offsets->data(), arena.get());

View File

@ -123,9 +123,10 @@ DataTypePtr FunctionArrayReduceInRanges::getReturnTypeImpl(const ColumnsWithType
}
ColumnPtr FunctionArrayReduceInRanges::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
ColumnPtr FunctionArrayReduceInRanges::executeImpl(
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
IAggregateFunction & agg_func = *aggregate_function;
const IAggregateFunction & agg_func = *aggregate_function;
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
@ -257,9 +258,9 @@ ColumnPtr FunctionArrayReduceInRanges::executeImpl(const ColumnsWithTypeAndName
agg_func.destroy(places[j]);
});
auto * true_func = &agg_func;
const auto * true_func = &agg_func;
/// Unnest consecutive trailing -State combinators
while (auto * func = typeid_cast<AggregateFunctionState *>(true_func))
while (const auto * func = typeid_cast<const AggregateFunctionState *>(true_func))
true_func = func->getNestedFunction().get();
/// Pre-aggregate to the initial level

View File

@ -44,6 +44,7 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override;
private:
/// TODO Rewrite with FunctionBuilder.
mutable AggregateFunctionPtr aggregate_function;
};
@ -89,7 +90,7 @@ DataTypePtr FunctionInitializeAggregation::getReturnTypeImpl(const ColumnsWithTy
ColumnPtr FunctionInitializeAggregation::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
IAggregateFunction & agg_func = *aggregate_function;
const IAggregateFunction & agg_func = *aggregate_function;
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
const size_t num_arguments_columns = arguments.size() - 1;
@ -138,9 +139,9 @@ ColumnPtr FunctionInitializeAggregation::executeImpl(const ColumnsWithTypeAndNam
});
{
auto * that = &agg_func;
const auto * that = &agg_func;
/// Unnest consecutive trailing -State combinators
while (auto * func = typeid_cast<AggregateFunctionState *>(that))
while (const auto * func = typeid_cast<const AggregateFunctionState *>(that))
that = func->getNestedFunction().get();
that->addBatch(input_rows_count, places.data(), 0, aggregate_arguments, arena.get());
}

View File

@ -651,7 +651,7 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns
aggregate_functions_instructions[i].arguments = aggregate_columns[i].data();
aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i];
auto * that = aggregate_functions[i];
const auto * that = aggregate_functions[i];
/// Unnest consecutive trailing -State combinators
while (const auto * func = typeid_cast<const AggregateFunctionState *>(that))
that = func->getNestedFunction().get();

View File

@ -959,7 +959,7 @@ public:
using AggregateColumns = std::vector<ColumnRawPtrs>;
using AggregateColumnsData = std::vector<ColumnAggregateFunction::Container *>;
using AggregateColumnsConstData = std::vector<const ColumnAggregateFunction::Container *>;
using AggregateFunctionsPlainPtrs = std::vector<IAggregateFunction *>;
using AggregateFunctionsPlainPtrs = std::vector<const IAggregateFunction *>;
/// Process one block. Return false if the processing should be aborted (with group_by_overflow_mode = 'break').
bool executeOnBlock(const Block & block, AggregatedDataVariants & result,

View File

@ -3,7 +3,6 @@
#include <Columns/ColumnTuple.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/materializeBlock.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>

View File

@ -1742,7 +1742,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
&& processing_stage == QueryProcessingStage::FetchColumns
&& query_analyzer->hasAggregation()
&& (query_analyzer->aggregates().size() == 1)
&& typeid_cast<AggregateFunctionCount *>(query_analyzer->aggregates()[0].function.get());
&& typeid_cast<const AggregateFunctionCount *>(query_analyzer->aggregates()[0].function.get());
if (optimize_trivial_count)
{
@ -1766,7 +1766,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
if (num_rows)
{
AggregateFunctionCount & agg_count = static_cast<AggregateFunctionCount &>(*func);
const AggregateFunctionCount & agg_count = static_cast<const AggregateFunctionCount &>(*func);
/// We will process it up to "WithMergeableState".
std::vector<char> state(agg_count.sizeOfData());

View File

@ -68,7 +68,7 @@ int main(int argc, char ** argv)
using namespace DB;
using Key = UInt64;
using Value = std::vector<IAggregateFunction*>;
using Value = std::vector<const IAggregateFunction*>;
size_t n = argc < 2 ? 10000000 : std::stol(argv[1]);
//size_t m = std::stol(argv[2]);

View File

@ -5,7 +5,7 @@ namespace DB
{
class IAggregateFunction;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
}

View File

@ -215,7 +215,9 @@ WindowTransform::WindowTransform(const Block & input_header_,
}
workspace.argument_columns.assign(f.argument_names.size(), nullptr);
workspace.window_function_impl = aggregate_function->asWindowFunction();
/// Currently we have slightly wrong mixup of the interfaces of Window and Aggregate functions.
workspace.window_function_impl = dynamic_cast<IWindowFunction *>(const_cast<IAggregateFunction *>(aggregate_function.get()));
if (!workspace.window_function_impl)
{
workspace.aggregate_function_state.reset(
@ -1343,13 +1345,12 @@ struct WindowFunction
{
std::string name;
WindowFunction(const std::string & name_, const DataTypes & argument_types_,
const Array & parameters_)
WindowFunction(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_)
: IAggregateFunctionHelper<WindowFunction>(argument_types_, parameters_)
, name(name_)
{}
IWindowFunction * asWindowFunction() override { return this; }
bool isOnlyWindowFunction() const override { return true; }
[[noreturn]] void fail() const
{

View File

@ -1,20 +0,0 @@
#!/usr/bin/env bash
export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
function check()
{
$CLICKHOUSE_CLIENT -q "SELECT id, $1(s) FROM ( SELECT number % 10 as id, uniqState(number) as s FROM ( SELECT number FROM system.numbers LIMIT 100 ) GROUP BY number ) GROUP BY id" 2>&1 | grep -v -P '^(Received exception from server|Code: 43)' ||:
}
stateFunctions=("uniqState" "uniqExactState" "uniqHLL12State" "uniqCombinedState" "uniqUpToState") # "uniqThetaState" not tested because its availability depends on compilation options
for i1 in "${stateFunctions[@]}"
do
check "$i1"
done

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1,22 @@
SELECT uniqExact(x) FROM (SELECT uniqState(number) AS x FROM numbers(100));
SELECT uniqExact(x) FROM (SELECT uniqState(number) AS x FROM numbers(1000));
SELECT hex(toString(uniqExactState(x))) FROM (SELECT uniqState(number) AS x FROM numbers(1000));
SELECT hex(toString(uniqExactState(x))) FROM (SELECT quantileState(number) AS x FROM numbers(1000));
SELECT toTypeName(uniqExactState(x)) FROM (SELECT quantileState(number) AS x FROM numbers(1000));
SELECT toTypeName(initializeAggregation('uniqExact', 0));
SELECT toTypeName(initializeAggregation('uniqExactState', 0));
SELECT toTypeName(initializeAggregation('uniqExactState', initializeAggregation('quantileState', 0)));
SELECT hex(toString(initializeAggregation('quantileState', 0)));
SELECT toTypeName(initializeAggregation('sumState', initializeAggregation('quantileState', 0))); -- { serverError 43 }
SELECT toTypeName(initializeAggregation('anyState', initializeAggregation('quantileState', 0)));
SELECT toTypeName(initializeAggregation('anyState', initializeAggregation('uniqState', 0)));
SELECT hex(toString(initializeAggregation('uniqState', initializeAggregation('uniqState', 0))));
SELECT hex(toString(initializeAggregation('uniqState', initializeAggregation('quantileState', 0))));
SELECT hex(toString(initializeAggregation('anyLastState', initializeAggregation('uniqState', 0))));
SELECT hex(toString(initializeAggregation('anyState', initializeAggregation('uniqState', 0))));
SELECT hex(toString(initializeAggregation('maxState', initializeAggregation('uniqState', 0)))); -- { serverError 43 }
SELECT hex(toString(initializeAggregation('uniqExactState', initializeAggregation('uniqState', 0))));
SELECT finalizeAggregation(initializeAggregation('uniqExactState', initializeAggregation('uniqState', 0)));
SELECT toTypeName(quantileState(x)) FROM (SELECT uniqState(number) AS x FROM numbers(1000)); -- { serverError 43 }
SELECT hex(toString(quantileState(x))) FROM (SELECT uniqState(number) AS x FROM numbers(1000)); -- { serverError 43 }
SELECT hex(toString(anyState(x))), hex(toString(any(x))) FROM (SELECT uniqState(number) AS x FROM numbers(1000)) FORMAT Vertical;