mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Fix trash
This commit is contained in:
parent
406285dad1
commit
e275fc9c29
@ -1,7 +1,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionArray.h>
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionBitwise.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/AggregateFunctionBoundingRatio.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,11 +1,9 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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())
|
||||
|
@ -17,10 +17,10 @@
|
||||
#include <Common/CurrentThread.h>
|
||||
|
||||
#include <Poco/String.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,12 +1,10 @@
|
||||
#include <AggregateFunctions/AggregateFunctionIf.h>
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
#include "AggregateFunctionNull.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
|
@ -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(
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionMaxIntersections.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -1,12 +1,11 @@
|
||||
#include <AggregateFunctions/AggregateFunctionOrFill.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
|
@ -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);
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/AggregateFunctionRankCorrelation.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
|
||||
@ -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);
|
||||
|
@ -1,12 +1,10 @@
|
||||
#include <AggregateFunctions/AggregateFunctionResample.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionRetention.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -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.",
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionSum.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionSumCount.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user