diff --git a/dbms/scripts/test_uniq_functions.sh b/dbms/scripts/test_uniq_functions.sh index 9a4b6f20433..f7e2083610b 100755 --- a/dbms/scripts/test_uniq_functions.sh +++ b/dbms/scripts/test_uniq_functions.sh @@ -6,6 +6,6 @@ do do n=$(( 10**p * i )) echo -n "$n " - clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(number) from numbers($n);" + clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(15)(number) from numbers($n);" done done diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp index 77b6c9cfb97..6b63a719b8f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -130,9 +130,6 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory) factory.registerFunction("uniqExact", createAggregateFunctionUniq>); - - factory.registerFunction("uniqCombined", - createAggregateFunctionUniq>); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h index 140928959a3..980d62b40ec 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h @@ -22,7 +22,6 @@ #include #include -#include #include @@ -124,46 +123,6 @@ struct AggregateFunctionUniqExactData static String getName() { return "uniqExact"; } }; -template -struct AggregateFunctionUniqCombinedData -{ - using Key = UInt32; - using Set = CombinedCardinalityEstimator< - Key, - HashSet>, - 16, - 14, - 17, - TrivialHash, - UInt32, - HyperLogLogBiasEstimator, - HyperLogLogMode::FullFeatured>; - - Set set; - - static String getName() { return "uniqCombined"; } -}; - -template <> -struct AggregateFunctionUniqCombinedData -{ - using Key = UInt64; - using Set = CombinedCardinalityEstimator< - Key, - HashSet>, - 16, - 14, - 17, - TrivialHash, - UInt64, - HyperLogLogBiasEstimator, - HyperLogLogMode::FullFeatured>; - - Set set; - - static String getName() { return "uniqCombined"; } -}; - namespace detail { @@ -199,39 +158,6 @@ template <> struct AggregateFunctionUniqTraits } }; -/** Hash function for uniqCombined. - */ -template struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(T x) { return static_cast(intHash64(x)); } -}; - -template <> struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(UInt128 x) - { - return sipHash64(x); - } -}; - -template <> struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(Float32 x) - { - UInt64 res = ext::bit_cast(x); - return static_cast(intHash64(res)); - } -}; - -template <> struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(Float64 x) - { - UInt64 res = ext::bit_cast(x); - return static_cast(intHash64(res)); - } -}; - /** The structure for the delegation work to add one element to the `uniq` aggregate functions. * Used for partial specialization to add strings. @@ -255,19 +181,6 @@ struct OneAdder data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } } - else if constexpr (std::is_same_v>) - { - if constexpr (!std::is_same_v) - { - const auto & value = static_cast &>(column).getData()[row_num]; - data.set.insert(AggregateFunctionUniqCombinedTraits::hash(value)); - } - else - { - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); - } - } else if constexpr (std::is_same_v>) { if constexpr (!std::is_same_v) @@ -387,5 +300,4 @@ public: const char * getHeaderFilePath() const override { return __FILE__; } }; - } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp new file mode 100644 index 00000000000..cdaf5b90c70 --- /dev/null +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -0,0 +1,90 @@ +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ARGUMENT_OUT_OF_BOUND; +} + +namespace +{ + +AggregateFunctionPtr createAggregateFunctionUniqCombined( + const std::string & name, const DataTypes & argument_types, const Array & params) +{ + UInt8 precision = 17; /// default value - must correlate with default ctor of |AggregateFunctionUniqCombinedData| + + if (!params.empty()) + { + if (params.size() != 1) + throw Exception( + "Aggregate function " + name + " requires one parameter or less.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + UInt64 precision_param = applyVisitor(FieldVisitorConvertToNumber(), params[0]); + + // This range is hardcoded into |AggregateFunctionUniqCombinedData| + if (precision_param > 20 || precision_param < 12) + throw Exception( + "Parameter for aggregate function " + name + "is out or range: [12, 20].", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + precision = precision_param; + } + + if (argument_types.empty()) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + /// 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 = !isAllArgumentsContiguousInMemory(argument_types); + + if (argument_types.size() == 1) + { + const IDataType & argument_type = *argument_types[0]; + + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], precision)); + + WhichDataType which(argument_type); + if (res) + return res; + else if (which.isDate()) + return std::make_shared>(precision); + else if (which.isDateTime()) + return std::make_shared>(precision); + else if (which.isStringOrFixedString()) + return std::make_shared>(precision); + else if (which.isUUID()) + return std::make_shared>(precision); + else if (which.isTuple()) + { + if (use_exact_hash_function) + return std::make_shared>(argument_types, precision); + else + return std::make_shared>(argument_types, precision); + } + } + + /// "Variadic" method also works as a fallback generic case for single argument. + if (use_exact_hash_function) + return std::make_shared>(argument_types, precision); + else + return std::make_shared>(argument_types, precision); +} + +} // namespace + +void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory) +{ + factory.registerFunction("uniqCombined", createAggregateFunctionUniqCombined); +} + +} // namespace DB diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h new file mode 100644 index 00000000000..8d8a7c6745d --- /dev/null +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -0,0 +1,429 @@ +#pragma once + +#include + +#include +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace detail +{ + /** Hash function for uniqCombined. + */ + template + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(T x) + { + return static_cast(intHash64(x)); + } + }; + + template <> + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(UInt128 x) + { + return sipHash64(x); + } + }; + + template <> + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(Float32 x) + { + UInt64 res = ext::bit_cast(x); + return static_cast(intHash64(res)); + } + }; + + template <> + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(Float64 x) + { + UInt64 res = ext::bit_cast(x); + return static_cast(intHash64(res)); + } + }; + +} // namespace detail + + +template +struct __attribute__((__packed__)) AggregateFunctionUniqCombinedDataWithKey +{ + template + using Set = CombinedCardinalityEstimator>, + 16, + K - 3, + K, + TrivialHash, + Key, + HyperLogLogBiasEstimator, + HyperLogLogMode::FullFeatured>; + + mutable UInt8 inited = 0; + union + { + Set<12> set_12; + Set<13> set_13; + Set<14> set_14; + Set<15> set_15; + Set<16> set_16; + Set<17> set_17; + Set<18> set_18; + Set<19> set_19; + Set<20> set_20; + }; + + AggregateFunctionUniqCombinedDataWithKey() : set_17() {} + + ~AggregateFunctionUniqCombinedDataWithKey() + { + switch (inited) + { + case 12: + set_12.~CombinedCardinalityEstimator(); + break; + case 13: + set_13.~CombinedCardinalityEstimator(); + break; + case 14: + set_14.~CombinedCardinalityEstimator(); + break; + case 15: + set_15.~CombinedCardinalityEstimator(); + break; + case 16: + set_16.~CombinedCardinalityEstimator(); + break; + case 0: + case 17: + set_17.~CombinedCardinalityEstimator(); + break; + case 18: + set_18.~CombinedCardinalityEstimator(); + break; + case 19: + set_19.~CombinedCardinalityEstimator(); + break; + case 20: + set_20.~CombinedCardinalityEstimator(); + break; + } + } + + void init(UInt8 precision) const + { + if (inited || precision == 17) + return; + + // TODO: assert "inited == precision" + + set_17.~CombinedCardinalityEstimator(); + + switch (precision) + { + case 12: + new (&set_12) Set<12>; + break; + case 13: + new (&set_13) Set<13>; + break; + case 14: + new (&set_14) Set<14>; + break; + case 15: + new (&set_15) Set<15>; + break; + case 16: + new (&set_16) Set<16>; + break; + case 18: + new (&set_18) Set<18>; + break; + case 19: + new (&set_19) Set<19>; + break; + case 20: + new (&set_20) Set<20>; + break; + } + inited = precision; + } + +#define SET_METHOD(method) \ + switch (inited) \ + { \ + case 12: \ + set_12.method; \ + break; \ + case 13: \ + set_13.method; \ + break; \ + case 14: \ + set_14.method; \ + break; \ + case 15: \ + set_15.method; \ + break; \ + case 16: \ + set_16.method; \ + break; \ + case 17: \ + set_17.method; \ + break; \ + case 18: \ + set_18.method; \ + break; \ + case 19: \ + set_19.method; \ + break; \ + case 20: \ + set_20.method; \ + break; \ + } + +#define SET_RETURN_METHOD(method) \ + switch (inited) \ + { \ + case 12: \ + return set_12.method; \ + case 13: \ + return set_13.method; \ + case 14: \ + return set_14.method; \ + case 15: \ + return set_15.method; \ + case 16: \ + return set_16.method; \ + case 18: \ + return set_18.method; \ + case 19: \ + return set_19.method; \ + case 20: \ + return set_20.method; \ + case 17: \ + default: \ + return set_17.method; \ + } + + void insert(Key value, UInt8 precision) + { + init(precision); + SET_METHOD(insert(value)); + } + + void merge(const AggregateFunctionUniqCombinedDataWithKey & rhs, UInt8 precision) + { + init(precision); + switch (inited) + { + case 12: + set_12.merge(rhs.set_12); + break; + case 13: + set_13.merge(rhs.set_13); + break; + case 14: + set_14.merge(rhs.set_14); + break; + case 15: + set_15.merge(rhs.set_15); + break; + case 16: + set_16.merge(rhs.set_16); + break; + case 17: + set_17.merge(rhs.set_17); + break; + case 18: + set_18.merge(rhs.set_18); + break; + case 19: + set_19.merge(rhs.set_19); + break; + case 20: + set_20.merge(rhs.set_20); + break; + } + } + + void write(DB::WriteBuffer & out, UInt8 precision) const + { + init(precision); + SET_METHOD(write(out)); + } + + void read(DB::ReadBuffer & in, UInt8 precision) + { + init(precision); + SET_METHOD(read(in)); + } + + UInt32 size(UInt8 precision) const + { + init(precision); + SET_RETURN_METHOD(size()); + } + +#undef SET_METHOD +#undef SET_RETURN_METHOD +}; + + +template +struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +{ +}; + + +template <> +struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +{ +}; + + +template +class AggregateFunctionUniqCombined final + : public IAggregateFunctionDataHelper, AggregateFunctionUniqCombined> +{ +private: + const UInt8 precision; + +public: + explicit AggregateFunctionUniqCombined(UInt8 precision) : precision(precision) {} + + String getName() const override + { + return "uniqCombined"; + } + + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override + { + if constexpr (!std::is_same_v) + { + const auto & value = static_cast &>(*columns[0]).getData()[row_num]; + this->data(place).insert(detail::AggregateFunctionUniqCombinedTraits::hash(value), precision); + } + else + { + StringRef value = columns[0]->getDataAt(row_num); + this->data(place).insert(CityHash_v1_0_2::CityHash64(value.data, value.size), precision); + } + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs), precision); + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + this->data(place).write(buf, precision); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override + { + this->data(place).read(buf, precision); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + static_cast(to).getData().push_back(this->data(place).size(precision)); + } + + const char * getHeaderFilePath() const override + { + return __FILE__; + } +}; + +/** For multiple arguments. To compute, hashes them. + * You can pass multiple arguments as is; You can also pass one argument - a tuple. + * But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples. + */ +template +class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper, + AggregateFunctionUniqCombinedVariadic> +{ +private: + size_t num_args = 0; + UInt8 precision; + +public: + AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, UInt8 precision) : precision(precision) + { + if (argument_is_tuple) + num_args = typeid_cast(*arguments[0]).getElements().size(); + else + num_args = arguments.size(); + } + + String getName() const override + { + return "uniqCombined"; + } + + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override + { + this->data(place).insert(typename AggregateFunctionUniqCombinedData::Set<12>::value_type( + UniqVariadicHash::apply(num_args, columns, row_num)), + precision); + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs), precision); + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + this->data(place).write(buf, precision); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override + { + this->data(place).read(buf, precision); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + static_cast(to).getData().push_back(this->data(place).size(precision)); + } + + const char * getHeaderFilePath() const override + { + return __FILE__; + } +}; + +} // namespace DB diff --git a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp index 3517ad57a73..800beda1d53 100644 --- a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -21,6 +21,7 @@ void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &); void registerAggregateFunctionSum(AggregateFunctionFactory &); void registerAggregateFunctionSumMap(AggregateFunctionFactory &); void registerAggregateFunctionsUniq(AggregateFunctionFactory &); +void registerAggregateFunctionUniqCombined(AggregateFunctionFactory &); void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &); void registerAggregateFunctionTopK(AggregateFunctionFactory &); void registerAggregateFunctionsBitwise(AggregateFunctionFactory &); @@ -55,6 +56,7 @@ void registerAggregateFunctions() registerAggregateFunctionSum(factory); registerAggregateFunctionSumMap(factory); registerAggregateFunctionsUniq(factory); + registerAggregateFunctionUniqCombined(factory); registerAggregateFunctionUniqUpTo(factory); registerAggregateFunctionTopK(factory); registerAggregateFunctionsBitwise(factory); diff --git a/dbms/tests/performance/test_hits/test_hits.xml b/dbms/tests/performance/test_hits/test_hits.xml index c9e30227ff0..eea308fdd64 100644 --- a/dbms/tests/performance/test_hits/test_hits.xml +++ b/dbms/tests/performance/test_hits/test_hits.xml @@ -86,8 +86,8 @@ PageCharset тоже почти всегда непуст, но его сред SELECT uniq(UserID) FROM test.hits SETTINGS max_threads = 1 SELECT uniq(UserID) FROM test.hits -SELECT uniqCombined(UserID) FROM test.hits SETTINGS max_threads = 1 -SELECT uniqCombined(UserID) FROM test.hits +SELECT uniqCombined(15)(UserID) FROM test.hits SETTINGS max_threads = 1 +SELECT uniqCombined(15)(UserID) FROM test.hits SELECT uniqExact(UserID) FROM test.hits SETTINGS max_threads = 1 SELECT uniqExact(UserID) FROM test.hits diff --git a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql index 53b5ec0001b..b3d82b71685 100644 --- a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql +++ b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql @@ -20,20 +20,20 @@ SELECT uniqHLL12(dummy) FROM remote('127.0.0.{2,3}', system.one); /* uniqCombined */ -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT uniqCombined(dummy) FROM remote('127.0.0.{2,3}', system.one); +SELECT uniqCombined(15)(dummy) FROM remote('127.0.0.{2,3}', system.one); diff --git a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql index 2b24e68910c..a94ec6e8f0a 100644 --- a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql +++ b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql @@ -1,12 +1,12 @@ -SELECT +SELECT uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), - uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)), + uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z)) FROM ( - SELECT + SELECT number % 10 AS x, intDiv(number, 10) % 10 AS y, toString(intDiv(number, 100) % 10) AS z @@ -16,14 +16,14 @@ FROM SELECT k, uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), - uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)), + uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z)), count() AS c FROM ( - SELECT + SELECT (number + 0x8ffcbd8257219a26) * 0x66bb3430c06d2353 % 131 AS k, number % 10 AS x, intDiv(number, 10) % 10 AS y, diff --git a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql index e10f665ad02..ba097ada98b 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql @@ -28,7 +28,7 @@ SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a > 0; SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a < 0; SELECT (uniq(a), uniq(b), uniq(c)), - (uniqCombined(a), uniqCombined(b), uniqCombined(c)), + (uniqCombined(15)(a), uniqCombined(15)(b), uniqCombined(15)(c)), (uniqExact(a), uniqExact(b), uniqExact(c)), (uniqHLL12(a), uniqHLL12(b), uniqHLL12(c)) FROM (SELECT * FROM test.decimal ORDER BY a); diff --git a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql index fd3fde7636d..0f860948a3a 100644 --- a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql +++ b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql @@ -1,3 +1,3 @@ SELECT RegionID, uniqHLL12(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; -SELECT RegionID, uniqCombined(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; +SELECT RegionID, uniqCombined(15)(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT abs(uniq(WatchID) - uniqExact(WatchID)) FROM test.hits; diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index fd0bb213933..491a710fe5b 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -146,7 +146,7 @@ The result is determinate (it doesn't depend on the order of query processing). This function provides excellent accuracy even for data sets with extremely high cardinality (over 10 billion elements). It is recommended for default use. -## uniqCombined(x) +## uniqCombined(HLL_precision)(x) Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments. diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index 1c700851ef0..d42f216c6ea 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -156,7 +156,7 @@ GROUP BY timeslot Данная функция обеспечивает отличную точность даже для множеств огромной кардинальности (10B+ элементов) и рекомендуется к использованию по умолчанию. -## uniqCombined(x) +## uniqCombined(HLL_precision)(x) Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей.