mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #3406 from abyss7/3958/many_templates
Move-away "uniqCombined" as a separate aggregated function
This commit is contained in:
commit
89e3c5c6bd
@ -1,4 +1,4 @@
|
||||
#!/usr/bin/python3.4
|
||||
#!/usr/bin/python3
|
||||
# -*- coding: utf-8 -*-
|
||||
|
||||
import sys
|
||||
|
@ -130,9 +130,6 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory)
|
||||
|
||||
factory.registerFunction("uniqExact",
|
||||
createAggregateFunctionUniq<true, AggregateFunctionUniqExactData, AggregateFunctionUniqExactData<String>>);
|
||||
|
||||
factory.registerFunction("uniqCombined",
|
||||
createAggregateFunctionUniq<false, AggregateFunctionUniqCombinedData, AggregateFunctionUniqCombinedData<UInt64>>);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -22,7 +22,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/UniqCombinedBiasData.h>
|
||||
#include <AggregateFunctions/UniqVariadicHash.h>
|
||||
|
||||
|
||||
@ -124,46 +123,6 @@ struct AggregateFunctionUniqExactData<String>
|
||||
static String getName() { return "uniqExact"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedData
|
||||
{
|
||||
using Key = UInt32;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
TrivialHash,
|
||||
UInt32,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
HyperLogLogMode::FullFeatured>;
|
||||
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedData<String>
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
TrivialHash,
|
||||
UInt64,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
HyperLogLogMode::FullFeatured>;
|
||||
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
|
||||
namespace detail
|
||||
{
|
||||
@ -199,39 +158,6 @@ template <> struct AggregateFunctionUniqTraits<Float64>
|
||||
}
|
||||
};
|
||||
|
||||
/** Hash function for uniqCombined.
|
||||
*/
|
||||
template <typename T> struct AggregateFunctionUniqCombinedTraits
|
||||
{
|
||||
static UInt32 hash(T x) { return static_cast<UInt32>(intHash64(x)); }
|
||||
};
|
||||
|
||||
template <> struct AggregateFunctionUniqCombinedTraits<UInt128>
|
||||
{
|
||||
static UInt32 hash(UInt128 x)
|
||||
{
|
||||
return sipHash64(x);
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct AggregateFunctionUniqCombinedTraits<Float32>
|
||||
{
|
||||
static UInt32 hash(Float32 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(intHash64(res));
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct AggregateFunctionUniqCombinedTraits<Float64>
|
||||
{
|
||||
static UInt32 hash(Float64 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(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<Data, AggregateFunctionUniqCombinedData<T>>)
|
||||
{
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
data.set.insert(AggregateFunctionUniqCombinedTraits<T>::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<Data, AggregateFunctionUniqExactData<T>>)
|
||||
{
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
@ -387,5 +300,4 @@ public:
|
||||
const char * getHeaderFilePath() const override { return __FILE__; }
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
127
dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp
Normal file
127
dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp
Normal file
@ -0,0 +1,127 @@
|
||||
#include <AggregateFunctions/AggregateFunctionUniqCombined.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
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
|
||||
{
|
||||
template <UInt8 K>
|
||||
struct WithK
|
||||
{
|
||||
template <typename T>
|
||||
using AggregateFunction = AggregateFunctionUniqCombined<T, K>;
|
||||
|
||||
template <bool is_exact, bool argument_is_tuple>
|
||||
using AggregateFunctionVariadic = AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K>;
|
||||
};
|
||||
|
||||
template <UInt8 K>
|
||||
AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types)
|
||||
{
|
||||
/// We use exact hash function if the arguments are not contiguous in memory, because only exact hash function has 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<WithK<K>::template AggregateFunction>(*argument_types[0]));
|
||||
|
||||
WhichDataType which(argument_type);
|
||||
if (res)
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunction<DataTypeDate::FieldType>>();
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunction<DataTypeDateTime::FieldType>>();
|
||||
else if (which.isStringOrFixedString())
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunction<String>>();
|
||||
else if (which.isUUID())
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunction<DataTypeUUID::FieldType>>();
|
||||
else if (which.isTuple())
|
||||
{
|
||||
if (use_exact_hash_function)
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<true, true>>(argument_types);
|
||||
else
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<false, true>>(argument_types);
|
||||
}
|
||||
}
|
||||
|
||||
/// "Variadic" method also works as a fallback generic case for a single argument.
|
||||
if (use_exact_hash_function)
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<true, false>>(argument_types);
|
||||
else
|
||||
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<false, false>>(argument_types);
|
||||
}
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionUniqCombined(
|
||||
const std::string & name, const DataTypes & argument_types, const Array & params)
|
||||
{
|
||||
/// log2 of the number of cells in HyperLogLog.
|
||||
/// Reasonable default value, selected to be comparable in quality with "uniq" aggregate function.
|
||||
UInt8 precision = 17;
|
||||
|
||||
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<UInt64>(), params[0]);
|
||||
|
||||
// This range is hardcoded below
|
||||
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);
|
||||
|
||||
switch (precision)
|
||||
{
|
||||
case 12:
|
||||
return createAggregateFunctionWithK<12>(argument_types);
|
||||
case 13:
|
||||
return createAggregateFunctionWithK<13>(argument_types);
|
||||
case 14:
|
||||
return createAggregateFunctionWithK<14>(argument_types);
|
||||
case 15:
|
||||
return createAggregateFunctionWithK<15>(argument_types);
|
||||
case 16:
|
||||
return createAggregateFunctionWithK<16>(argument_types);
|
||||
case 17:
|
||||
return createAggregateFunctionWithK<17>(argument_types);
|
||||
case 18:
|
||||
return createAggregateFunctionWithK<18>(argument_types);
|
||||
case 19:
|
||||
return createAggregateFunctionWithK<19>(argument_types);
|
||||
case 20:
|
||||
return createAggregateFunctionWithK<20>(argument_types);
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("uniqCombined", createAggregateFunctionUniqCombined);
|
||||
}
|
||||
|
||||
} // namespace DB
|
229
dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h
Normal file
229
dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h
Normal file
@ -0,0 +1,229 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/CombinedCardinalityEstimator.h>
|
||||
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/UniqCombinedBiasData.h>
|
||||
#include <AggregateFunctions/UniqVariadicHash.h>
|
||||
|
||||
#include <ext/bit_cast.h>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace detail
|
||||
{
|
||||
/** Hash function for uniqCombined.
|
||||
*/
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedTraits
|
||||
{
|
||||
static UInt32 hash(T x)
|
||||
{
|
||||
return static_cast<UInt32>(intHash64(x));
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedTraits<UInt128>
|
||||
{
|
||||
static UInt32 hash(UInt128 x)
|
||||
{
|
||||
return sipHash64(x);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedTraits<Float32>
|
||||
{
|
||||
static UInt32 hash(Float32 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(intHash64(res));
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedTraits<Float64>
|
||||
{
|
||||
static UInt32 hash(Float64 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(intHash64(res));
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace detail
|
||||
|
||||
|
||||
template <typename Key, UInt8 K>
|
||||
struct AggregateFunctionUniqCombinedDataWithKey
|
||||
{
|
||||
// TODO(ilezhankin): pre-generate values for |UniqCombinedBiasData|,
|
||||
// at the moment gen-bias-data.py script doesn't work.
|
||||
|
||||
// We want to migrate from |HashSet| to |HyperLogLogCounter| when the sizes in memory become almost equal.
|
||||
// The size per element in |HashSet| is sizeof(Key)*2 bytes, and the overall size of |HyperLogLogCounter| is 2^K * 6 bits.
|
||||
// For Key=UInt32 we can calculate: 2^X * 4 * 2 ≤ 2^(K-3) * 6 ⇒ X ≤ K-4.
|
||||
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, HashTableGrower<>>, 16, K - 4, K, TrivialHash, Key>;
|
||||
|
||||
Set set;
|
||||
};
|
||||
|
||||
template <typename Key>
|
||||
struct AggregateFunctionUniqCombinedDataWithKey<Key, 17>
|
||||
{
|
||||
using Set = CombinedCardinalityEstimator<Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
||||
16,
|
||||
13,
|
||||
17,
|
||||
TrivialHash,
|
||||
Key,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
HyperLogLogMode::FullFeatured>;
|
||||
|
||||
Set set;
|
||||
};
|
||||
|
||||
|
||||
template <typename T, UInt8 K>
|
||||
struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey<UInt32, K>
|
||||
{
|
||||
};
|
||||
|
||||
|
||||
template <UInt8 K>
|
||||
struct AggregateFunctionUniqCombinedData<String, K> : public AggregateFunctionUniqCombinedDataWithKey<UInt64, K>
|
||||
{
|
||||
};
|
||||
|
||||
|
||||
template <typename T, UInt8 K>
|
||||
class AggregateFunctionUniqCombined final
|
||||
: public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K>, AggregateFunctionUniqCombined<T, K>>
|
||||
{
|
||||
public:
|
||||
String getName() const override
|
||||
{
|
||||
return "uniqCombined";
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||
{
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
|
||||
this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits<T>::hash(value));
|
||||
}
|
||||
else
|
||||
{
|
||||
StringRef value = columns[0]->getDataAt(row_num);
|
||||
this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
|
||||
}
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||
{
|
||||
this->data(place).set.merge(this->data(rhs).set);
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).set.write(buf);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
||||
{
|
||||
this->data(place).set.read(buf);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
|
||||
}
|
||||
|
||||
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 <bool is_exact, bool argument_is_tuple, UInt8 K>
|
||||
class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K>,
|
||||
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K>>
|
||||
{
|
||||
private:
|
||||
size_t num_args = 0;
|
||||
|
||||
public:
|
||||
explicit AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments)
|
||||
{
|
||||
if (argument_is_tuple)
|
||||
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
|
||||
else
|
||||
num_args = arguments.size();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return "uniqCombined";
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||
{
|
||||
this->data(place).set.insert(typename AggregateFunctionUniqCombinedData<UInt64, K>::Set::value_type(
|
||||
UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num)));
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||
{
|
||||
this->data(place).set.merge(this->data(rhs).set);
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).set.write(buf);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
||||
{
|
||||
this->data(place).set.read(buf);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
|
||||
}
|
||||
|
||||
const char * getHeaderFilePath() const override
|
||||
{
|
||||
return __FILE__;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace DB
|
@ -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);
|
||||
|
@ -86,8 +86,8 @@ PageCharset тоже почти всегда непуст, но его сред
|
||||
<query>SELECT uniq(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniq(UserID) FROM test.hits</query>
|
||||
<!-- 43. Разные алгоритмы вычисления кардинальности. -->
|
||||
<query>SELECT uniqCombined(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniqCombined(UserID) FROM test.hits</query>
|
||||
<query>SELECT uniqCombined(17)(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniqCombined(17)(UserID) FROM test.hits</query>
|
||||
<!-- 44. Разные алгоритмы вычисления кардинальности. -->
|
||||
<query>SELECT uniqExact(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniqExact(UserID) FROM test.hits</query>
|
||||
|
@ -259,6 +259,162 @@
|
||||
31 53948
|
||||
35 53931
|
||||
36 53982
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
7 1
|
||||
9 1
|
||||
11 1
|
||||
14 1
|
||||
17 1
|
||||
19 1
|
||||
20 2
|
||||
26 1
|
||||
31 1
|
||||
35 1
|
||||
36 1
|
||||
0 162
|
||||
1 162
|
||||
3 162
|
||||
6 162
|
||||
7 163
|
||||
9 163
|
||||
10 81
|
||||
11 163
|
||||
13 162
|
||||
14 162
|
||||
17 162
|
||||
19 162
|
||||
20 162
|
||||
21 162
|
||||
22 162
|
||||
26 162
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 54708
|
||||
1 53721
|
||||
3 53226
|
||||
6 54532
|
||||
7 52275
|
||||
9 53417
|
||||
10 26931
|
||||
11 54428
|
||||
13 53409
|
||||
14 53188
|
||||
17 55120
|
||||
19 54123
|
||||
20 53293
|
||||
21 53928
|
||||
22 53827
|
||||
26 53920
|
||||
31 53763
|
||||
35 54635
|
||||
36 53155
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
7 1
|
||||
9 1
|
||||
11 1
|
||||
14 1
|
||||
17 1
|
||||
19 1
|
||||
20 2
|
||||
26 1
|
||||
31 1
|
||||
35 1
|
||||
36 1
|
||||
0 162
|
||||
1 162
|
||||
3 162
|
||||
6 162
|
||||
7 163
|
||||
9 163
|
||||
10 81
|
||||
11 163
|
||||
13 162
|
||||
14 162
|
||||
17 162
|
||||
19 162
|
||||
20 162
|
||||
21 162
|
||||
22 162
|
||||
26 162
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 53901
|
||||
1 54056
|
||||
3 53999
|
||||
6 54129
|
||||
7 54213
|
||||
9 53853
|
||||
10 26975
|
||||
11 54064
|
||||
13 53963
|
||||
14 53997
|
||||
17 54129
|
||||
19 53923
|
||||
20 53958
|
||||
21 54117
|
||||
22 54150
|
||||
26 54047
|
||||
31 53948
|
||||
35 53931
|
||||
36 53982
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
7 1
|
||||
9 1
|
||||
11 1
|
||||
14 1
|
||||
17 1
|
||||
19 1
|
||||
20 2
|
||||
26 1
|
||||
31 1
|
||||
35 1
|
||||
36 1
|
||||
0 162
|
||||
1 162
|
||||
3 162
|
||||
6 162
|
||||
7 163
|
||||
9 163
|
||||
10 81
|
||||
11 163
|
||||
13 162
|
||||
14 162
|
||||
17 162
|
||||
19 162
|
||||
20 162
|
||||
21 162
|
||||
22 162
|
||||
26 162
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 54054
|
||||
1 54054
|
||||
3 54054
|
||||
6 54054
|
||||
7 54053
|
||||
9 54053
|
||||
10 27027
|
||||
11 54055
|
||||
13 54054
|
||||
14 54054
|
||||
17 54054
|
||||
19 54053
|
||||
20 54053
|
||||
21 54054
|
||||
22 54053
|
||||
26 54053
|
||||
31 54054
|
||||
35 54054
|
||||
36 54054
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
@ -311,6 +467,162 @@
|
||||
0.043 54150
|
||||
0.037 54047
|
||||
0.071 53963
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.143 1
|
||||
0.056 1
|
||||
0.048 2
|
||||
0.083 1
|
||||
0.25 1
|
||||
0.1 1
|
||||
0.028 1
|
||||
0.027 1
|
||||
0.031 1
|
||||
0.067 1
|
||||
0.037 1
|
||||
0.045 162
|
||||
0.125 163
|
||||
0.5 162
|
||||
0.05 162
|
||||
0.143 162
|
||||
0.091 81
|
||||
0.056 162
|
||||
0.048 162
|
||||
0.083 163
|
||||
0.25 162
|
||||
1 162
|
||||
0.1 163
|
||||
0.028 162
|
||||
0.027 162
|
||||
0.031 162
|
||||
0.067 162
|
||||
0.043 162
|
||||
0.037 162
|
||||
0.071 162
|
||||
0.045 53928
|
||||
0.125 52275
|
||||
0.5 53721
|
||||
0.05 54123
|
||||
0.143 54532
|
||||
0.091 26931
|
||||
0.056 55120
|
||||
0.048 53293
|
||||
0.083 54428
|
||||
0.25 53226
|
||||
1 54708
|
||||
0.1 53417
|
||||
0.028 54635
|
||||
0.027 53155
|
||||
0.031 53763
|
||||
0.067 53188
|
||||
0.043 53827
|
||||
0.037 53920
|
||||
0.071 53409
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.143 1
|
||||
0.056 1
|
||||
0.048 2
|
||||
0.083 1
|
||||
0.25 1
|
||||
0.1 1
|
||||
0.028 1
|
||||
0.027 1
|
||||
0.031 1
|
||||
0.067 1
|
||||
0.037 1
|
||||
0.045 162
|
||||
0.125 163
|
||||
0.5 162
|
||||
0.05 162
|
||||
0.143 162
|
||||
0.091 81
|
||||
0.056 162
|
||||
0.048 162
|
||||
0.083 163
|
||||
0.25 162
|
||||
1 162
|
||||
0.1 163
|
||||
0.028 162
|
||||
0.027 162
|
||||
0.031 162
|
||||
0.067 162
|
||||
0.043 162
|
||||
0.037 162
|
||||
0.071 162
|
||||
0.045 54117
|
||||
0.125 54213
|
||||
0.5 54056
|
||||
0.05 53923
|
||||
0.143 54129
|
||||
0.091 26975
|
||||
0.056 54129
|
||||
0.048 53958
|
||||
0.083 54064
|
||||
0.25 53999
|
||||
1 53901
|
||||
0.1 53853
|
||||
0.028 53931
|
||||
0.027 53982
|
||||
0.031 53948
|
||||
0.067 53997
|
||||
0.043 54150
|
||||
0.037 54047
|
||||
0.071 53963
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.143 1
|
||||
0.056 1
|
||||
0.048 2
|
||||
0.083 1
|
||||
0.25 1
|
||||
0.1 1
|
||||
0.028 1
|
||||
0.027 1
|
||||
0.031 1
|
||||
0.067 1
|
||||
0.037 1
|
||||
0.045 162
|
||||
0.125 163
|
||||
0.5 162
|
||||
0.05 162
|
||||
0.143 162
|
||||
0.091 81
|
||||
0.056 162
|
||||
0.048 162
|
||||
0.083 163
|
||||
0.25 162
|
||||
1 162
|
||||
0.1 163
|
||||
0.028 162
|
||||
0.027 162
|
||||
0.031 162
|
||||
0.067 162
|
||||
0.043 162
|
||||
0.037 162
|
||||
0.071 162
|
||||
0.045 54054
|
||||
0.125 54053
|
||||
0.5 54054
|
||||
0.05 54053
|
||||
0.143 54054
|
||||
0.091 27027
|
||||
0.056 54054
|
||||
0.048 54053
|
||||
0.083 54055
|
||||
0.25 54054
|
||||
1 54054
|
||||
0.1 54053
|
||||
0.028 54054
|
||||
0.027 54054
|
||||
0.031 54054
|
||||
0.067 54054
|
||||
0.043 54053
|
||||
0.037 54053
|
||||
0.071 54054
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
@ -363,6 +675,162 @@
|
||||
0.037 54047
|
||||
0.1 53853
|
||||
1 53901
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
0.048 2
|
||||
0.083 1
|
||||
0.125 1
|
||||
0.031 1
|
||||
0.143 1
|
||||
0.028 1
|
||||
0.067 1
|
||||
0.027 1
|
||||
0.056 1
|
||||
0.037 1
|
||||
0.1 1
|
||||
0.5 162
|
||||
0.05 162
|
||||
0.25 162
|
||||
0.048 162
|
||||
0.091 81
|
||||
0.043 162
|
||||
0.071 162
|
||||
0.083 163
|
||||
0.125 163
|
||||
0.031 162
|
||||
0.143 162
|
||||
0.028 162
|
||||
0.067 162
|
||||
0.045 162
|
||||
0.027 162
|
||||
0.056 162
|
||||
0.037 162
|
||||
0.1 163
|
||||
1 162
|
||||
0.5 53721
|
||||
0.05 54123
|
||||
0.25 53226
|
||||
0.048 53293
|
||||
0.091 26931
|
||||
0.043 53827
|
||||
0.071 53409
|
||||
0.083 54428
|
||||
0.125 52275
|
||||
0.031 53763
|
||||
0.143 54532
|
||||
0.028 54635
|
||||
0.067 53188
|
||||
0.045 53928
|
||||
0.027 53155
|
||||
0.056 55120
|
||||
0.037 53920
|
||||
0.1 53417
|
||||
1 54708
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
0.048 2
|
||||
0.083 1
|
||||
0.125 1
|
||||
0.031 1
|
||||
0.143 1
|
||||
0.028 1
|
||||
0.067 1
|
||||
0.027 1
|
||||
0.056 1
|
||||
0.037 1
|
||||
0.1 1
|
||||
0.5 162
|
||||
0.05 162
|
||||
0.25 162
|
||||
0.048 162
|
||||
0.091 81
|
||||
0.043 162
|
||||
0.071 162
|
||||
0.083 163
|
||||
0.125 163
|
||||
0.031 162
|
||||
0.143 162
|
||||
0.028 162
|
||||
0.067 162
|
||||
0.045 162
|
||||
0.027 162
|
||||
0.056 162
|
||||
0.037 162
|
||||
0.1 163
|
||||
1 162
|
||||
0.5 54056
|
||||
0.05 53923
|
||||
0.25 53999
|
||||
0.048 53958
|
||||
0.091 26975
|
||||
0.043 54150
|
||||
0.071 53963
|
||||
0.083 54064
|
||||
0.125 54213
|
||||
0.031 53948
|
||||
0.143 54129
|
||||
0.028 53931
|
||||
0.067 53997
|
||||
0.045 54117
|
||||
0.027 53982
|
||||
0.056 54129
|
||||
0.037 54047
|
||||
0.1 53853
|
||||
1 53901
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
0.048 2
|
||||
0.083 1
|
||||
0.125 1
|
||||
0.031 1
|
||||
0.143 1
|
||||
0.028 1
|
||||
0.067 1
|
||||
0.027 1
|
||||
0.056 1
|
||||
0.037 1
|
||||
0.1 1
|
||||
0.5 162
|
||||
0.05 162
|
||||
0.25 162
|
||||
0.048 162
|
||||
0.091 81
|
||||
0.043 162
|
||||
0.071 162
|
||||
0.083 163
|
||||
0.125 163
|
||||
0.031 162
|
||||
0.143 162
|
||||
0.028 162
|
||||
0.067 162
|
||||
0.045 162
|
||||
0.027 162
|
||||
0.056 162
|
||||
0.037 162
|
||||
0.1 163
|
||||
1 162
|
||||
0.5 54054
|
||||
0.05 54053
|
||||
0.25 54054
|
||||
0.048 54053
|
||||
0.091 27027
|
||||
0.043 54053
|
||||
0.071 54054
|
||||
0.083 54055
|
||||
0.125 54053
|
||||
0.031 54054
|
||||
0.143 54054
|
||||
0.028 54054
|
||||
0.067 54054
|
||||
0.045 54054
|
||||
0.027 54054
|
||||
0.056 54054
|
||||
0.037 54053
|
||||
0.1 54053
|
||||
1 54054
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -415,4 +883,163 @@
|
||||
31 54074
|
||||
35 54153
|
||||
36 53999
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
7 1
|
||||
9 1
|
||||
11 1
|
||||
14 1
|
||||
17 1
|
||||
19 1
|
||||
20 2
|
||||
26 1
|
||||
31 1
|
||||
35 1
|
||||
36 1
|
||||
0 162
|
||||
1 162
|
||||
3 162
|
||||
6 162
|
||||
7 163
|
||||
9 163
|
||||
10 81
|
||||
11 163
|
||||
13 162
|
||||
14 162
|
||||
17 162
|
||||
19 162
|
||||
20 162
|
||||
21 162
|
||||
22 162
|
||||
26 162
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 52613
|
||||
1 54468
|
||||
3 53824
|
||||
6 54441
|
||||
7 54543
|
||||
9 51908
|
||||
10 26964
|
||||
11 54013
|
||||
13 53178
|
||||
14 54113
|
||||
17 54662
|
||||
19 54697
|
||||
20 53279
|
||||
21 55301
|
||||
22 53693
|
||||
26 53873
|
||||
31 55200
|
||||
35 54808
|
||||
36 53051
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
7 1
|
||||
9 1
|
||||
11 1
|
||||
14 1
|
||||
17 1
|
||||
19 1
|
||||
20 2
|
||||
26 1
|
||||
31 1
|
||||
35 1
|
||||
36 1
|
||||
0 162
|
||||
1 162
|
||||
3 162
|
||||
6 162
|
||||
7 163
|
||||
9 163
|
||||
10 81
|
||||
11 163
|
||||
13 162
|
||||
14 162
|
||||
17 162
|
||||
19 162
|
||||
20 162
|
||||
21 162
|
||||
22 162
|
||||
26 162
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 54195
|
||||
1 54086
|
||||
3 54127
|
||||
6 54173
|
||||
7 53969
|
||||
9 54257
|
||||
10 26985
|
||||
11 53880
|
||||
13 54105
|
||||
14 54043
|
||||
17 54176
|
||||
19 53913
|
||||
20 54088
|
||||
21 53991
|
||||
22 54112
|
||||
26 54136
|
||||
31 54074
|
||||
35 54153
|
||||
36 53999
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
7 1
|
||||
9 1
|
||||
11 1
|
||||
14 1
|
||||
17 1
|
||||
19 1
|
||||
20 2
|
||||
26 1
|
||||
31 1
|
||||
35 1
|
||||
36 1
|
||||
0 162
|
||||
1 162
|
||||
3 162
|
||||
6 162
|
||||
7 163
|
||||
9 163
|
||||
10 81
|
||||
11 163
|
||||
13 162
|
||||
14 162
|
||||
17 162
|
||||
19 162
|
||||
20 162
|
||||
21 162
|
||||
22 162
|
||||
26 162
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 54054
|
||||
1 54054
|
||||
3 54054
|
||||
6 54054
|
||||
7 54054
|
||||
9 54054
|
||||
10 27027
|
||||
11 54055
|
||||
13 54054
|
||||
14 54054
|
||||
17 54054
|
||||
19 54054
|
||||
20 54054
|
||||
21 54054
|
||||
22 54054
|
||||
26 54054
|
||||
31 54054
|
||||
35 54054
|
||||
36 54054
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
|
@ -24,16 +24,70 @@ SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS 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(12)(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(12)(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(12)(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(17)(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(17)(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(17)(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(20)(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(20)(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(20)(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(12)(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(12)(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(12)(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(17)(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(17)(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(17)(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(20)(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(20)(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(20)(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(12)(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(12)(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(12)(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(17)(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(17)(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(17)(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(20)(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(20)(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(20)(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(12)(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(12)(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(12)(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(17)(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(17)(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(17)(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(20)(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(20)(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(20)(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(12)(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
||||
SELECT uniqCombined(17)(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
||||
SELECT uniqCombined(20)(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
@ -1,11 +1,11 @@
|
||||
10 10 100 100 1000 1000 10 10 100 100 1000 1000 10 10 101 101 1006 1006 10 10 100 100 1000 1000 6 6 6 6 6 6
|
||||
17 10 10 100 100 610 610 10 10 100 100 610 610 10 10 101 101 616 616 10 10 100 100 610 610 6 6 6 6 6 6 766
|
||||
52 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 611 611 10 10 100 100 608 608 6 6 6 6 6 6 766
|
||||
5 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 608 608 10 10 100 100 609 609 6 6 6 6 6 6 765
|
||||
9 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 618 618 10 10 100 100 608 608 6 6 6 6 6 6 765
|
||||
13 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 610 610 10 10 100 100 607 607 6 6 6 6 6 6 765
|
||||
46 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 611 611 10 10 100 100 607 607 6 6 6 6 6 6 765
|
||||
48 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 617 617 10 10 100 100 609 609 6 6 6 6 6 6 765
|
||||
50 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765
|
||||
54 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 615 615 10 10 100 100 609 609 6 6 6 6 6 6 765
|
||||
56 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765
|
||||
10 10 100 100 1000 1000 10 10 100 100 1000 1000 10 10 100 100 1000 1000 10 10 101 101 1006 1006 10 10 100 100 1000 1000 6 6 6 6 6 6
|
||||
17 10 10 100 100 610 610 10 10 100 100 610 610 10 10 100 100 610 610 10 10 101 101 616 616 10 10 100 100 610 610 6 6 6 6 6 6 766
|
||||
52 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 611 611 10 10 100 100 608 608 6 6 6 6 6 6 766
|
||||
5 10 10 100 100 609 609 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 608 608 10 10 100 100 609 609 6 6 6 6 6 6 765
|
||||
9 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 618 618 10 10 100 100 608 608 6 6 6 6 6 6 765
|
||||
13 10 10 100 100 607 607 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 610 610 10 10 100 100 607 607 6 6 6 6 6 6 765
|
||||
46 10 10 100 100 607 607 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 611 611 10 10 100 100 607 607 6 6 6 6 6 6 765
|
||||
48 10 10 100 100 609 609 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 617 617 10 10 100 100 609 609 6 6 6 6 6 6 765
|
||||
50 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765
|
||||
54 10 10 100 100 609 609 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 615 615 10 10 100 100 609 609 6 6 6 6 6 6 765
|
||||
56 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765
|
||||
|
@ -1,12 +1,13 @@
|
||||
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(17)(x), uniqCombined(17)((x)), uniqCombined(17)(x, y), uniqCombined(17)((x, y)), uniqCombined(17)(x, y, z), uniqCombined(17)((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
|
||||
@ -17,13 +18,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(17)(x), uniqCombined(17)((x)), uniqCombined(17)(x, y), uniqCombined(17)((x, y)), uniqCombined(17)(x, y, z), uniqCombined(17)((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,
|
||||
|
@ -8,7 +8,7 @@
|
||||
0.0000 0.00000000 0.00000000
|
||||
25.5000 8.49999999 5.10000000
|
||||
-25.5000 -8.49999999 -5.10000000
|
||||
(101,101,101) (101,101,101) (101,101,101) (102,100,101)
|
||||
(101,101,101) (101,101,101) (101,101,101) (101,101,101) (102,100,101)
|
||||
5 5 5
|
||||
10 10 10
|
||||
-50.0000 -50.0000 -16.66666666 -16.66666666 -10.00000000 -10.00000000
|
||||
|
@ -29,6 +29,7 @@ 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(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)),
|
||||
(uniqExact(a), uniqExact(b), uniqExact(c)),
|
||||
(uniqHLL12(a), uniqHLL12(b), uniqHLL12(c))
|
||||
FROM (SELECT * FROM test.decimal ORDER BY a);
|
||||
|
@ -283,15 +283,17 @@ 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.
|
||||
|
||||
A combination of three algorithms is used: array, hash table and [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) with an error correction table. The memory consumption is several times smaller than for the `uniq` function, and the accuracy is several times higher. Performance is slightly lower than for the `uniq` function, but sometimes it can be even higher than it, such as with distributed queries that transmit a large number of aggregation states over the network. The maximum state size is 96 KiB (HyperLogLog of 217 6-bit cells).
|
||||
A combination of three algorithms is used: array, hash table and [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) with an error correction table. For small number of distinct elements, the array is used. When the set size becomes larger the hash table is used, while it is smaller than HyperLogLog data structure. For larger number of elements, the HyperLogLog is used, and it will occupy fixed amount of memory.
|
||||
|
||||
The result is determinate (it doesn't depend on the order of query processing).
|
||||
The parameter "HLL_precision" is the base-2 logarithm of the number of cells in HyperLogLog. You can omit the parameter (omit first parens). The default value is 17, that is effectively 96 KiB of space (2^17 cells of 6 bits each). The memory consumption is several times smaller than for the `uniq` function, and the accuracy is several times higher. Performance is slightly lower than for the `uniq` function, but sometimes it can be even higher than it, such as with distributed queries that transmit a large number of aggregation states over the network.
|
||||
|
||||
The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error will increase for high-cardinality data sets (200M+ elements), and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements).
|
||||
The result is deterministic (it doesn't depend on the order of query processing).
|
||||
|
||||
The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error for large sets (200 million elements and more) will become larger than theoretical value due to poor choice of hash function.
|
||||
|
||||
## uniqHLL12(x)
|
||||
|
||||
|
@ -284,15 +284,17 @@ GROUP BY timeslot
|
||||
|
||||
Функция обеспечивает высокую точность даже для множеств с высокой кардинальностью (более 10 миллиардов элементов). Рекомендуется для использования по умолчанию.
|
||||
|
||||
## uniqCombined(x)
|
||||
## uniqCombined(HLL_precision)(x)
|
||||
|
||||
Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей.
|
||||
|
||||
Используется комбинация трех алгоритмов: массив, хэш-таблица и [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) с таблицей коррекции погрешности. Расход памяти в несколько раз меньше, чем у функции `uniq`, а точность в несколько раз выше. Скорость работы чуть ниже, чем у функции `uniq`, но иногда может быть даже выше - в случае распределённых запросов, в которых по сети передаётся большое количество состояний агрегации. Максимальный размер состояния составляет 96 KiB (HyperLogLog из 217 6-битовых ячеек).
|
||||
Используется комбинация трёх алгоритмов: массив, хэш-таблица и [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) с таблицей коррекции погрешности. Для небольшого количества различных значений используется массив; при увеличении количества значений, используется хэш таблица, до тех пор, пока её размер меньше размера HyperLogLog структуры. При дальнейшем увеличении количества значений, используется HyperLogLog структура, имеющая фиксированный размер в памяти.
|
||||
|
||||
Параметр HLL_precision - логарифм по основанию 2 от количества ячеек в HyperLogLog. Параметер можно не указывать (для этого, опустите первую пару скобок). По-умолчанию - 17. При использовании параметра по-умолчанию, расход памяти в несколько раз меньше, чем у функции `uniq`, а точность в несколько раз выше. Скорость работы чуть ниже, чем у функции `uniq`, но иногда может быть даже выше - в случае распределённых запросов, в которых по сети передаётся большое количество состояний агрегации. Каждая ячейка имеет размер 6 бит, что даёт 96 KiB для размера HyperLogLog структуры.
|
||||
|
||||
Результат детерминирован (не зависит от порядка выполнения запроса).
|
||||
|
||||
Функция `uniqCombined` является хорошим выбором по умолчанию для подсчёта количества различных значений, но стоит иметь ввиду что для множеств большой кардинальности (200M+) ошибка оценки будет только расти и для множеств огромной кардинальности (1B+ элементов) функция возвращает результат с очень большой неточностью.
|
||||
Функция `uniqCombined` является хорошим выбором по умолчанию для подсчёта количества различных значений, но стоит иметь ввиду что для множеств большой кардинальности (200 миллионов различных элементов и больше) ошибка оценки становится существенно больше расчётной из-за недостаточно хорошего выбора хэш-функции.
|
||||
|
||||
## uniqHLL12(x)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user