mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
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:
commit
7616216f10
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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,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);
|
||||
|
@ -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
|
||||
{
|
||||
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
13
src/AggregateFunctions/IAggregateFunction.cpp
Normal file
13
src/AggregateFunctions/IAggregateFunction.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -61,6 +61,7 @@ SRCS(
|
||||
AggregateFunctionUniqUpTo.cpp
|
||||
AggregateFunctionWelchTTest.cpp
|
||||
AggregateFunctionWindowFunnel.cpp
|
||||
IAggregateFunction.cpp
|
||||
UniqCombinedBiasData.cpp
|
||||
UniqVariadicHash.cpp
|
||||
parseAggregateFunctionParameters.cpp
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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());
|
||||
|
@ -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
|
||||
|
@ -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());
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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,
|
||||
|
@ -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>
|
||||
|
@ -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());
|
||||
|
@ -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]);
|
||||
|
@ -5,7 +5,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
|
||||
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
21
tests/queries/0_stateless/01890_state_of_state.reference
Normal file
21
tests/queries/0_stateless/01890_state_of_state.reference
Normal file
File diff suppressed because one or more lines are too long
22
tests/queries/0_stateless/01890_state_of_state.sql
Normal file
22
tests/queries/0_stateless/01890_state_of_state.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user