From 17e3a4ff1f21ebd342882362f4f8cc5881c3a088 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 10 Feb 2017 12:02:10 +0300 Subject: [PATCH] Implemented aggregate function 'count' with Nullable arguments to count number of not-NULL values [#CLICKHOUSE-2833]. --- .../AggregateFunctionCount.h | 134 +++++++++++++++++- .../INullaryAggregateFunction.h | 10 +- .../IUnaryAggregateFunction.h | 2 +- .../AggregateFunctionCount.cpp | 7 + .../AggregateFunctionFactory.cpp | 18 ++- .../00425_count_nullable.reference | 12 ++ .../0_stateless/00425_count_nullable.sql | 16 +++ 7 files changed, 181 insertions(+), 18 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00425_count_nullable.reference create mode 100644 dbms/tests/queries/0_stateless/00425_count_nullable.sql diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionCount.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionCount.h index 0b2b5fcd086..e979666fea0 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionCount.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionCount.h @@ -3,8 +3,11 @@ #include #include +#include #include +#include +#include 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 { 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(); } - void addImpl(AggregateDataPtr place) const { ++data(place).count; @@ -55,11 +60,128 @@ public: static_cast(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 +{ +public: + String getName() const override { return "count"; } + + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } + + void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena * arena) const + { + data(place).count += !static_cast(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(to).getData().push_back(data(place).count); + } +}; + + +/// Count number of calls with all arguments are not NULL. +class AggregateFunctionCountNotNullVariadic final : public IAggregateFunctionHelper +{ +public: + String getName() const override { return "count"; } + + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } + + 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(*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(*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(to).getData().push_back(data(place).count); + } + +private: + enum { MAX_ARGS = 8 }; + size_t number_of_arguments = 0; + std::array is_nullable; /// Plain array is better than std::vector due to one indirection less. +}; + } diff --git a/dbms/include/DB/AggregateFunctions/INullaryAggregateFunction.h b/dbms/include/DB/AggregateFunctions/INullaryAggregateFunction.h index 4af5bd87d9e..7acbd48c61f 100644 --- a/dbms/include/DB/AggregateFunctions/INullaryAggregateFunction.h +++ b/dbms/include/DB/AggregateFunctions/INullaryAggregateFunction.h @@ -7,7 +7,7 @@ namespace DB { -/** Интерфейс для ноль-арных агрегатных функций. Это, например, агрегатная функция count. +/** Interface for aggregate functions taking zero number of arguments. For example, this is 'count' aggregate function. */ template class INullaryAggregateFunction : public IAggregateFunctionHelper @@ -17,15 +17,15 @@ private: const Derived & getDerived() const { return static_cast(*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; */ }; diff --git a/dbms/include/DB/AggregateFunctions/IUnaryAggregateFunction.h b/dbms/include/DB/AggregateFunctions/IUnaryAggregateFunction.h index 81a6d90c4e3..6b7c1ece5e0 100644 --- a/dbms/include/DB/AggregateFunctions/IUnaryAggregateFunction.h +++ b/dbms/include/DB/AggregateFunctions/IUnaryAggregateFunction.h @@ -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); */ }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp b/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp index d9ec4fbee80..9287e2020b5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp @@ -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(); + return std::make_shared(); +} + } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 590c0d7ce3c..c35d5917fab 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -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(*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); diff --git a/dbms/tests/queries/0_stateless/00425_count_nullable.reference b/dbms/tests/queries/0_stateless/00425_count_nullable.reference new file mode 100644 index 00000000000..45d1e2fecc2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00425_count_nullable.reference @@ -0,0 +1,12 @@ +10 +10 +10 +10 +10 +10 +10 +9 +9 +9 +8 +0 diff --git a/dbms/tests/queries/0_stateless/00425_count_nullable.sql b/dbms/tests/queries/0_stateless/00425_count_nullable.sql new file mode 100644 index 00000000000..b90fc5e0f2f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00425_count_nullable.sql @@ -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);