mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Implemented aggregate function 'count' with Nullable arguments to count number of not-NULL values [#CLICKHOUSE-2833].
This commit is contained in:
parent
1ad4e2752e
commit
17e3a4ff1f
@ -3,8 +3,11 @@
|
||||
#include <DB/IO/VarInt.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <DB/AggregateFunctions/INullaryAggregateFunction.h>
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
#include <DB/Columns/ColumnNullable.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,24 +15,26 @@ namespace DB
|
||||
|
||||
struct AggregateFunctionCountData
|
||||
{
|
||||
UInt64 count;
|
||||
|
||||
AggregateFunctionCountData() : count(0) {}
|
||||
UInt64 count = 0;
|
||||
};
|
||||
|
||||
|
||||
/// Просто считает, сколько раз её вызвали
|
||||
/// Simply count number of calls.
|
||||
class AggregateFunctionCount final : public INullaryAggregateFunction<AggregateFunctionCountData, AggregateFunctionCount>
|
||||
{
|
||||
public:
|
||||
String getName() const override { return "count"; }
|
||||
|
||||
void setArguments(const DataTypes & arguments) override
|
||||
{
|
||||
/// You may pass some arguments. All of them are ignored.
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
|
||||
void addImpl(AggregateDataPtr place) const
|
||||
{
|
||||
++data(place).count;
|
||||
@ -55,11 +60,128 @@ public:
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(data(place).count);
|
||||
}
|
||||
|
||||
/// Для оптимизации
|
||||
/// May be used for optimization.
|
||||
void addDelta(AggregateDataPtr place, UInt64 x) const
|
||||
{
|
||||
data(place).count += x;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Simply count number of not-NULL values.
|
||||
class AggregateFunctionCountNotNullUnary final : public IUnaryAggregateFunction<AggregateFunctionCountData, AggregateFunctionCountNotNullUnary>
|
||||
{
|
||||
public:
|
||||
String getName() const override { return "count"; }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena * arena) const
|
||||
{
|
||||
data(place).count += !static_cast<const ColumnNullable &>(column).isNullAt(row_num);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->isNullable() && !argument->isNull())
|
||||
throw Exception("Not Nullable argument passed to aggregate function count", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
|
||||
{
|
||||
data(place).count += data(rhs).count;
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
writeVarUInt(data(place).count, buf);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
||||
{
|
||||
readVarUInt(data(place).count, buf);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(data(place).count);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Count number of calls with all arguments are not NULL.
|
||||
class AggregateFunctionCountNotNullVariadic final : public IAggregateFunctionHelper<AggregateFunctionCountData>
|
||||
{
|
||||
public:
|
||||
String getName() const override { return "count"; }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void setArguments(const DataTypes & arguments) override
|
||||
{
|
||||
number_of_arguments = arguments.size();
|
||||
|
||||
if (number_of_arguments == 1)
|
||||
throw Exception("Logical error: single argument is passed to AggregateFunctionCountNotNullVariadic", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (number_of_arguments > MAX_ARGS)
|
||||
throw Exception("Maximum number of arguments for aggregate function with Nullable types is " + toString(MAX_ARGS),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (size_t i = 0; i < number_of_arguments; ++i)
|
||||
is_nullable[i] = arguments[i]->isNullable() || arguments[i]->isNull();
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override
|
||||
{
|
||||
for (size_t i = 0; i < number_of_arguments; ++i)
|
||||
if (is_nullable[i] && static_cast<const ColumnNullable &>(*columns[i]).isNullAt(row_num))
|
||||
return;
|
||||
|
||||
++data(place).count;
|
||||
}
|
||||
|
||||
static void addFree(const IAggregateFunction * that, AggregateDataPtr place,
|
||||
const IColumn ** columns, size_t row_num, Arena * arena)
|
||||
{
|
||||
return static_cast<const AggregateFunctionCountNotNullVariadic &>(*that).add(place, columns, row_num, arena);
|
||||
}
|
||||
|
||||
AddFunc getAddressOfAddFunction() const override
|
||||
{
|
||||
return &addFree;
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
|
||||
{
|
||||
data(place).count += data(rhs).count;
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
writeVarUInt(data(place).count, buf);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
||||
{
|
||||
readVarUInt(data(place).count, buf);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(data(place).count);
|
||||
}
|
||||
|
||||
private:
|
||||
enum { MAX_ARGS = 8 };
|
||||
size_t number_of_arguments = 0;
|
||||
std::array<char, MAX_ARGS> is_nullable; /// Plain array is better than std::vector due to one indirection less.
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -7,7 +7,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Интерфейс для ноль-арных агрегатных функций. Это, например, агрегатная функция count.
|
||||
/** Interface for aggregate functions taking zero number of arguments. For example, this is 'count' aggregate function.
|
||||
*/
|
||||
template <typename T, typename Derived>
|
||||
class INullaryAggregateFunction : public IAggregateFunctionHelper<T>
|
||||
@ -17,15 +17,15 @@ private:
|
||||
const Derived & getDerived() const { return static_cast<const Derived &>(*this); }
|
||||
|
||||
public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
void setArguments(const DataTypes & arguments) override final
|
||||
/// By default, checks that number of arguments is zero. You could override if you would like to allow to have ignored arguments.
|
||||
void setArguments(const DataTypes & arguments) override
|
||||
{
|
||||
if (arguments.size() != 0)
|
||||
throw Exception("Passed " + toString(arguments.size()) + " arguments to nullary aggregate function " + this->getName(),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
/// Добавить значение.
|
||||
/// Accumulate a value.
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override final
|
||||
{
|
||||
getDerived().addImpl(place);
|
||||
@ -38,7 +38,7 @@ public:
|
||||
|
||||
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
|
||||
|
||||
/** Реализуйте это в классе-наследнике:
|
||||
/** Implement the following in descendant class:
|
||||
* void addImpl(AggregateDataPtr place) const;
|
||||
*/
|
||||
};
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
IAggregateFunction::AddFunc getAddressOfAddFunction() const override { return &addFree; }
|
||||
|
||||
/** Implement the following in descendant class:
|
||||
* void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const;
|
||||
* void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena * arena) const;
|
||||
* void setArgument(const DataTypePtr & argument);
|
||||
*/
|
||||
};
|
||||
|
@ -19,4 +19,11 @@ void registerAggregateFunctionCount(AggregateFunctionFactory & factory)
|
||||
factory.registerFunction("count", createAggregateFunctionCount, AggregateFunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionCountNotNull(const DataTypes & argument_types)
|
||||
{
|
||||
if (argument_types.size() == 1)
|
||||
return std::make_shared<AggregateFunctionCountNotNullUnary>();
|
||||
return std::make_shared<AggregateFunctionCountNotNullVariadic>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -54,6 +54,7 @@ AggregateFunctionPtr createAggregateFunctionState(AggregateFunctionPtr & nested)
|
||||
AggregateFunctionPtr createAggregateFunctionMerge(AggregateFunctionPtr & nested);
|
||||
AggregateFunctionPtr createAggregateFunctionNullUnary(AggregateFunctionPtr & nested);
|
||||
AggregateFunctionPtr createAggregateFunctionNullVariadic(AggregateFunctionPtr & nested);
|
||||
AggregateFunctionPtr createAggregateFunctionCountNotNull(const DataTypes & argument_types);
|
||||
|
||||
|
||||
AggregateFunctionFactory::AggregateFunctionFactory()
|
||||
@ -100,7 +101,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
bool has_nullable_types = false;
|
||||
for (const auto & arg_type : argument_types)
|
||||
{
|
||||
if (arg_type->isNullable())
|
||||
if (arg_type->isNullable() || arg_type->isNull())
|
||||
{
|
||||
has_nullable_types = true;
|
||||
break;
|
||||
@ -109,8 +110,13 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
|
||||
if (has_nullable_types)
|
||||
{
|
||||
DataTypes new_argument_types;
|
||||
new_argument_types.reserve(argument_types.size());
|
||||
/// Special case for 'count' function. It could be called with Nullable arguments
|
||||
/// - that means - count number of calls, when all arguments are not NULL.
|
||||
if (Poco::toLower(name) == "count")
|
||||
return createAggregateFunctionCountNotNull(argument_types);
|
||||
|
||||
DataTypes nested_argument_types;
|
||||
nested_argument_types.reserve(argument_types.size());
|
||||
|
||||
for (const auto & arg_type : argument_types)
|
||||
{
|
||||
@ -118,13 +124,13 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
{
|
||||
const DataTypeNullable & actual_type = static_cast<const DataTypeNullable &>(*arg_type.get());
|
||||
const DataTypePtr & nested_type = actual_type.getNestedType();
|
||||
new_argument_types.push_back(nested_type);
|
||||
nested_argument_types.push_back(nested_type);
|
||||
}
|
||||
else
|
||||
new_argument_types.push_back(arg_type);
|
||||
nested_argument_types.push_back(arg_type);
|
||||
}
|
||||
|
||||
AggregateFunctionPtr function = getImpl(name, new_argument_types, recursion_level);
|
||||
AggregateFunctionPtr function = getImpl(name, nested_argument_types, recursion_level);
|
||||
|
||||
if (argument_types.size() == 1)
|
||||
return createAggregateFunctionNullUnary(function);
|
||||
|
@ -0,0 +1,12 @@
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
9
|
||||
9
|
||||
9
|
||||
8
|
||||
0
|
16
dbms/tests/queries/0_stateless/00425_count_nullable.sql
Normal file
16
dbms/tests/queries/0_stateless/00425_count_nullable.sql
Normal file
@ -0,0 +1,16 @@
|
||||
SELECT count() FROM (SELECT number AS x FROM system.numbers LIMIT 10);
|
||||
|
||||
SELECT count(x) FROM (SELECT number AS x FROM system.numbers LIMIT 10);
|
||||
SELECT count(x, y) FROM (SELECT number AS x, number AS y FROM system.numbers LIMIT 10);
|
||||
|
||||
SELECT count(x) FROM (SELECT CAST(number AS Nullable(UInt64)) AS x FROM system.numbers LIMIT 10);
|
||||
SELECT count(x, y) FROM (SELECT CAST(number AS Nullable(UInt64)) AS x, number AS y FROM system.numbers LIMIT 10);
|
||||
SELECT count(x, y) FROM (SELECT number AS x, CAST(number AS Nullable(UInt64)) AS y FROM system.numbers LIMIT 10);
|
||||
SELECT count(x, y) FROM (SELECT CAST(number AS Nullable(UInt64)) AS x, CAST(number AS Nullable(UInt64)) AS y FROM system.numbers LIMIT 10);
|
||||
|
||||
SELECT count(x) FROM (SELECT nullIf(number, 5) AS x FROM system.numbers LIMIT 10);
|
||||
SELECT count(x, y) FROM (SELECT nullIf(number, 5) AS x, number AS y FROM system.numbers LIMIT 10);
|
||||
SELECT count(x, y) FROM (SELECT number AS x, nullIf(number, 3) AS y FROM system.numbers LIMIT 10);
|
||||
SELECT count(x, y) FROM (SELECT nullIf(number, 5) AS x, nullIf(number, 3) AS y FROM system.numbers LIMIT 10);
|
||||
|
||||
SELECT count(NULL);
|
Loading…
Reference in New Issue
Block a user