mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #23894 from pingyu/datasketches-uniq-again
Add uniqThetaSketch again
This commit is contained in:
commit
fb038c1985
2
contrib/datasketches-cpp
vendored
2
contrib/datasketches-cpp
vendored
@ -1 +1 @@
|
|||||||
Subproject commit f915d35b2de676683493c86c585141a1e1c83334
|
Subproject commit 7d73d7610db31d4e1ecde0fb3a7ee90ef371207f
|
@ -308,6 +308,8 @@ function run_tests
|
|||||||
01354_order_by_tuple_collate_const
|
01354_order_by_tuple_collate_const
|
||||||
01355_ilike
|
01355_ilike
|
||||||
01411_bayesian_ab_testing
|
01411_bayesian_ab_testing
|
||||||
|
01798_uniq_theta_sketch
|
||||||
|
01799_long_uniq_theta_sketch
|
||||||
collate
|
collate
|
||||||
collation
|
collation
|
||||||
_orc_
|
_orc_
|
||||||
|
@ -38,3 +38,4 @@ We recommend using this function in almost all scenarios.
|
|||||||
- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64)
|
- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64)
|
||||||
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12)
|
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12)
|
||||||
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
|
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
|
||||||
|
- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch)
|
||||||
|
@ -49,3 +49,4 @@ Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq
|
|||||||
- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64)
|
- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64)
|
||||||
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12)
|
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12)
|
||||||
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
|
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
|
||||||
|
- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch)
|
||||||
|
@ -23,3 +23,4 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, `
|
|||||||
- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq)
|
- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq)
|
||||||
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqcombined)
|
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqcombined)
|
||||||
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqhll12)
|
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqhll12)
|
||||||
|
- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch)
|
||||||
|
@ -37,3 +37,4 @@ We don’t recommend using this function. In most cases, use the [uniq](../../..
|
|||||||
- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq)
|
- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq)
|
||||||
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined)
|
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined)
|
||||||
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
|
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
|
||||||
|
- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch)
|
||||||
|
@ -0,0 +1,39 @@
|
|||||||
|
---
|
||||||
|
toc_priority: 195
|
||||||
|
---
|
||||||
|
|
||||||
|
# uniqThetaSketch {#agg_function-uniqthetasketch}
|
||||||
|
|
||||||
|
Calculates the approximate number of different argument values, using the [Theta Sketch Framework](https://datasketches.apache.org/docs/Theta/ThetaSketchFramework.html).
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
uniqThetaSketch(x[, ...])
|
||||||
|
```
|
||||||
|
|
||||||
|
**Arguments**
|
||||||
|
|
||||||
|
The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.
|
||||||
|
|
||||||
|
**Returned value**
|
||||||
|
|
||||||
|
- A [UInt64](../../../sql-reference/data-types/int-uint.md)-type number.
|
||||||
|
|
||||||
|
**Implementation details**
|
||||||
|
|
||||||
|
Function:
|
||||||
|
|
||||||
|
- Calculates a hash for all parameters in the aggregate, then uses it in calculations.
|
||||||
|
|
||||||
|
- Uses the [KMV](https://datasketches.apache.org/docs/Theta/InverseEstimate.html) algorithm to approximate the number of different argument values.
|
||||||
|
|
||||||
|
4096(2^12) 64-bit sketch are used. The size of the state is about 41 KB.
|
||||||
|
|
||||||
|
- The relative error is 3.125% (95% confidence), see the [relative error table](https://datasketches.apache.org/docs/Theta/ThetaErrorTable.html) for detail.
|
||||||
|
|
||||||
|
**See Also**
|
||||||
|
|
||||||
|
- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq)
|
||||||
|
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined)
|
||||||
|
- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64)
|
||||||
|
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12)
|
||||||
|
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
|
@ -132,6 +132,12 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory)
|
|||||||
|
|
||||||
factory.registerFunction("uniqExact",
|
factory.registerFunction("uniqExact",
|
||||||
{createAggregateFunctionUniq<true, AggregateFunctionUniqExactData, AggregateFunctionUniqExactData<String>>, properties});
|
{createAggregateFunctionUniq<true, AggregateFunctionUniqExactData, AggregateFunctionUniqExactData<String>>, properties});
|
||||||
|
|
||||||
|
#if USE_DATASKETCHES
|
||||||
|
factory.registerFunction("uniqThetaSketch",
|
||||||
|
{createAggregateFunctionUniq<AggregateFunctionUniqThetaSketchData, AggregateFunctionUniqThetaSketchData>, properties});
|
||||||
|
#endif
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -22,6 +22,7 @@
|
|||||||
|
|
||||||
#include <AggregateFunctions/UniquesHashSet.h>
|
#include <AggregateFunctions/UniquesHashSet.h>
|
||||||
#include <AggregateFunctions/IAggregateFunction.h>
|
#include <AggregateFunctions/IAggregateFunction.h>
|
||||||
|
#include <AggregateFunctions/ThetaSketchData.h>
|
||||||
#include <AggregateFunctions/UniqVariadicHash.h>
|
#include <AggregateFunctions/UniqVariadicHash.h>
|
||||||
|
|
||||||
|
|
||||||
@ -124,6 +125,19 @@ struct AggregateFunctionUniqExactData<String>
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/// uniqThetaSketch
|
||||||
|
#if USE_DATASKETCHES
|
||||||
|
|
||||||
|
struct AggregateFunctionUniqThetaSketchData
|
||||||
|
{
|
||||||
|
using Set = ThetaSketchData<UInt64>;
|
||||||
|
Set set;
|
||||||
|
|
||||||
|
static String getName() { return "uniqThetaSketch"; }
|
||||||
|
};
|
||||||
|
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace detail
|
namespace detail
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -187,6 +201,12 @@ struct OneAdder
|
|||||||
data.set.insert(key);
|
data.set.insert(key);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
#if USE_DATASKETCHES
|
||||||
|
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqThetaSketchData>)
|
||||||
|
{
|
||||||
|
data.set.insertOriginal(column.getDataAt(row_num));
|
||||||
|
}
|
||||||
|
#endif
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
119
src/AggregateFunctions/ThetaSketchData.h
Normal file
119
src/AggregateFunctions/ThetaSketchData.h
Normal file
@ -0,0 +1,119 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_DATASKETCHES
|
||||||
|
|
||||||
|
#include <boost/noncopyable.hpp>
|
||||||
|
#include <memory>
|
||||||
|
#include <theta_sketch.hpp> // Y_IGNORE
|
||||||
|
#include <theta_union.hpp> // Y_IGNORE
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
template <typename Key>
|
||||||
|
class ThetaSketchData : private boost::noncopyable
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
std::unique_ptr<datasketches::update_theta_sketch> sk_update;
|
||||||
|
std::unique_ptr<datasketches::theta_union> sk_union;
|
||||||
|
|
||||||
|
inline datasketches::update_theta_sketch * getSkUpdate()
|
||||||
|
{
|
||||||
|
if (!sk_update)
|
||||||
|
sk_update = std::make_unique<datasketches::update_theta_sketch>(datasketches::update_theta_sketch::builder().build());
|
||||||
|
return sk_update.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
inline datasketches::theta_union * getSkUnion()
|
||||||
|
{
|
||||||
|
if (!sk_union)
|
||||||
|
sk_union = std::make_unique<datasketches::theta_union>(datasketches::theta_union::builder().build());
|
||||||
|
return sk_union.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
public:
|
||||||
|
using value_type = Key;
|
||||||
|
|
||||||
|
ThetaSketchData() = default;
|
||||||
|
~ThetaSketchData() = default;
|
||||||
|
|
||||||
|
/// Insert original value without hash, as `datasketches::update_theta_sketch.update` will do the hash internal.
|
||||||
|
void insertOriginal(const StringRef & value)
|
||||||
|
{
|
||||||
|
getSkUpdate()->update(value.data, value.size);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Note that `datasketches::update_theta_sketch.update` will do the hash again.
|
||||||
|
void insert(Key value)
|
||||||
|
{
|
||||||
|
getSkUpdate()->update(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt64 size() const
|
||||||
|
{
|
||||||
|
if (sk_union)
|
||||||
|
return static_cast<UInt64>(sk_union->get_result().get_estimate());
|
||||||
|
else if (sk_update)
|
||||||
|
return static_cast<UInt64>(sk_update->get_estimate());
|
||||||
|
else
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
void merge(const ThetaSketchData & rhs)
|
||||||
|
{
|
||||||
|
datasketches::theta_union * u = getSkUnion();
|
||||||
|
|
||||||
|
if (sk_update)
|
||||||
|
{
|
||||||
|
u->update(*sk_update);
|
||||||
|
sk_update.reset(nullptr);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (rhs.sk_update)
|
||||||
|
u->update(*rhs.sk_update);
|
||||||
|
else if (rhs.sk_union)
|
||||||
|
u->update(rhs.sk_union->get_result());
|
||||||
|
}
|
||||||
|
|
||||||
|
/// You can only call for an empty object.
|
||||||
|
void read(DB::ReadBuffer & in)
|
||||||
|
{
|
||||||
|
datasketches::compact_theta_sketch::vector_bytes bytes;
|
||||||
|
readVectorBinary(bytes, in);
|
||||||
|
if (!bytes.empty())
|
||||||
|
{
|
||||||
|
auto sk = datasketches::compact_theta_sketch::deserialize(bytes.data(), bytes.size());
|
||||||
|
getSkUnion()->update(sk);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void write(DB::WriteBuffer & out) const
|
||||||
|
{
|
||||||
|
if (sk_update)
|
||||||
|
{
|
||||||
|
auto bytes = sk_update->compact().serialize();
|
||||||
|
writeVectorBinary(bytes, out);
|
||||||
|
}
|
||||||
|
else if (sk_union)
|
||||||
|
{
|
||||||
|
auto bytes = sk_union->get_result().serialize();
|
||||||
|
writeVectorBinary(bytes, out);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
datasketches::compact_theta_sketch::vector_bytes bytes;
|
||||||
|
writeVectorBinary(bytes, out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
#endif
|
@ -15,3 +15,4 @@
|
|||||||
#cmakedefine01 USE_GRPC
|
#cmakedefine01 USE_GRPC
|
||||||
#cmakedefine01 USE_STATS
|
#cmakedefine01 USE_STATS
|
||||||
#cmakedefine01 CLICKHOUSE_SPLIT_BINARY
|
#cmakedefine01 CLICKHOUSE_SPLIT_BINARY
|
||||||
|
#cmakedefine01 USE_DATASKETCHES
|
||||||
|
@ -12,7 +12,8 @@ namespace DB
|
|||||||
static bool isUniq(const ASTFunction & func)
|
static bool isUniq(const ASTFunction & func)
|
||||||
{
|
{
|
||||||
return func.name == "uniq" || func.name == "uniqExact" || func.name == "uniqHLL12"
|
return func.name == "uniq" || func.name == "uniqExact" || func.name == "uniqHLL12"
|
||||||
|| func.name == "uniqCombined" || func.name == "uniqCombined64";
|
|| func.name == "uniqCombined" || func.name == "uniqCombined64"
|
||||||
|
|| func.name == "uniqThetaSketch";
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Remove injective functions of one argument: replace with a child
|
/// Remove injective functions of one argument: replace with a child
|
||||||
|
230
tests/queries/0_stateless/01798_uniq_theta_sketch.reference
Normal file
230
tests/queries/0_stateless/01798_uniq_theta_sketch.reference
Normal file
@ -0,0 +1,230 @@
|
|||||||
|
uniqThetaSketch many agrs
|
||||||
|
10 10 100 100 1000 1000
|
||||||
|
17 10 10 100 100 610 610 766
|
||||||
|
52 10 10 100 100 608 608 766
|
||||||
|
5 10 10 100 100 608 608 765
|
||||||
|
9 10 10 100 100 608 608 765
|
||||||
|
13 10 10 100 100 607 607 765
|
||||||
|
46 10 10 100 100 607 607 765
|
||||||
|
48 10 10 100 100 609 609 765
|
||||||
|
50 10 10 100 100 608 608 765
|
||||||
|
54 10 10 100 100 609 609 765
|
||||||
|
56 10 10 100 100 608 608 765
|
||||||
|
uniqThetaSketch distinct
|
||||||
|
123
|
||||||
|
143
|
||||||
|
uniqThetaSketch arrays
|
||||||
|
2
|
||||||
|
3
|
||||||
|
3
|
||||||
|
uniqThetaSketch complex types
|
||||||
|
3
|
||||||
|
3
|
||||||
|
3
|
||||||
|
3
|
||||||
|
3
|
||||||
|
3
|
||||||
|
3
|
||||||
|
3
|
||||||
|
3
|
||||||
|
uniqThetaSketch decimals
|
||||||
|
(0,0,0)
|
||||||
|
(101,101,101)
|
||||||
|
uniqThetaSketch remove injective
|
||||||
|
SELECT uniqThetaSketch(x)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(x + y)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
number % 2 AS x,
|
||||||
|
number % 3 AS y
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(x)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(x)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(x)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(x)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(x)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(x + y)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
number % 2 AS x,
|
||||||
|
number % 3 AS y
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(-x)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(bitNot(x))
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(bitNot(-x))
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
SELECT uniqThetaSketch(-bitNot(-x))
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT number % 2 AS x
|
||||||
|
FROM numbers(10)
|
||||||
|
)
|
||||||
|
1000 1000
|
||||||
|
2014-06-01 1000 1000
|
||||||
|
1000 1000
|
||||||
|
2014-06-01 1000 1000
|
||||||
|
2014-06-01 0 0 7 7
|
||||||
|
2014-06-01 0 1 7 7
|
||||||
|
2014-06-01 0 2 7 7
|
||||||
|
2014-06-01 0 3 7 7
|
||||||
|
2014-06-01 0 4 7 7
|
||||||
|
2014-06-01 0 5 7 7
|
||||||
|
2014-06-01 0 6 7 7
|
||||||
|
2014-06-01 0 7 7 7
|
||||||
|
2014-06-01 0 8 7 7
|
||||||
|
2014-06-01 0 9 7 7
|
||||||
|
2014-06-01 1 10 7 7
|
||||||
|
2014-06-01 1 11 7 7
|
||||||
|
2014-06-01 1 12 7 7
|
||||||
|
2014-06-01 1 13 7 7
|
||||||
|
2014-06-01 1 14 7 7
|
||||||
|
2014-06-01 1 15 7 7
|
||||||
|
2014-06-01 1 16 7 7
|
||||||
|
2014-06-01 1 17 7 7
|
||||||
|
2014-06-01 1 18 7 7
|
||||||
|
2014-06-01 1 19 7 7
|
||||||
|
2014-06-01 2 20 7 7
|
||||||
|
2014-06-01 2 21 7 7
|
||||||
|
2014-06-01 2 22 7 7
|
||||||
|
2014-06-01 2 23 7 7
|
||||||
|
2014-06-01 2 24 7 7
|
||||||
|
2014-06-01 2 25 7 7
|
||||||
|
2014-06-01 2 26 7 7
|
||||||
|
2014-06-01 2 27 7 7
|
||||||
|
2014-06-01 2 28 7 7
|
||||||
|
2014-06-01 2 29 7 7
|
||||||
|
2014-06-01 3 30 7 7
|
||||||
|
2014-06-01 3 31 7 7
|
||||||
|
2014-06-01 3 32 7 7
|
||||||
|
2014-06-01 3 33 7 7
|
||||||
|
2014-06-01 3 34 7 7
|
||||||
|
2014-06-01 3 35 7 7
|
||||||
|
2014-06-01 3 36 7 7
|
||||||
|
2014-06-01 3 37 7 7
|
||||||
|
2014-06-01 3 38 7 7
|
||||||
|
2014-06-01 3 39 7 7
|
||||||
|
2014-06-01 4 40 7 7
|
||||||
|
2014-06-01 4 41 7 7
|
||||||
|
2014-06-01 4 42 7 7
|
||||||
|
2014-06-01 4 43 7 7
|
||||||
|
2014-06-01 4 44 7 7
|
||||||
|
2014-06-01 4 45 7 7
|
||||||
|
2014-06-01 4 46 7 7
|
||||||
|
2014-06-01 4 47 7 7
|
||||||
|
2014-06-01 4 48 7 7
|
||||||
|
2014-06-01 4 49 7 7
|
||||||
|
2014-06-01 5 50 7 7
|
||||||
|
2014-06-01 5 51 7 7
|
||||||
|
2014-06-01 5 52 7 7
|
||||||
|
2014-06-01 5 53 7 7
|
||||||
|
2014-06-01 5 54 7 7
|
||||||
|
2014-06-01 5 55 7 7
|
||||||
|
2014-06-01 5 56 7 7
|
||||||
|
2014-06-01 5 57 7 7
|
||||||
|
2014-06-01 5 58 7 7
|
||||||
|
2014-06-01 5 59 7 7
|
||||||
|
2014-06-01 6 60 7 7
|
||||||
|
2014-06-01 6 61 7 7
|
||||||
|
2014-06-01 6 62 7 7
|
||||||
|
2014-06-01 6 63 7 7
|
||||||
|
2014-06-01 6 64 7 7
|
||||||
|
2014-06-01 6 65 7 7
|
||||||
|
2014-06-01 6 66 7 7
|
||||||
|
2014-06-01 6 67 7 7
|
||||||
|
2014-06-01 6 68 7 7
|
||||||
|
2014-06-01 6 69 7 7
|
||||||
|
2014-06-01 7 70 7 7
|
||||||
|
2014-06-01 7 71 7 7
|
||||||
|
2014-06-01 7 72 7 7
|
||||||
|
2014-06-01 7 73 7 7
|
||||||
|
2014-06-01 7 74 7 7
|
||||||
|
2014-06-01 7 75 7 7
|
||||||
|
2014-06-01 7 76 7 7
|
||||||
|
2014-06-01 7 77 7 7
|
||||||
|
2014-06-01 7 78 7 7
|
||||||
|
2014-06-01 7 79 7 7
|
||||||
|
2014-06-01 8 80 7 7
|
||||||
|
2014-06-01 8 81 7 7
|
||||||
|
2014-06-01 8 82 7 7
|
||||||
|
2014-06-01 8 83 7 7
|
||||||
|
2014-06-01 8 84 7 7
|
||||||
|
2014-06-01 8 85 7 7
|
||||||
|
2014-06-01 8 86 7 7
|
||||||
|
2014-06-01 8 87 7 7
|
||||||
|
2014-06-01 8 88 7 7
|
||||||
|
2014-06-01 8 89 7 7
|
||||||
|
2014-06-01 9 90 7 7
|
||||||
|
2014-06-01 9 91 7 7
|
||||||
|
2014-06-01 9 92 7 7
|
||||||
|
2014-06-01 9 93 7 7
|
||||||
|
2014-06-01 9 94 7 7
|
||||||
|
2014-06-01 9 95 7 7
|
||||||
|
2014-06-01 9 96 7 7
|
||||||
|
2014-06-01 9 97 7 7
|
||||||
|
2014-06-01 9 98 7 7
|
||||||
|
2014-06-01 9 99 7 7
|
||||||
|
2014-06-01 0 7 7
|
||||||
|
2014-06-01 1 7 7
|
||||||
|
2014-06-01 2 7 7
|
||||||
|
2014-06-01 3 7 7
|
||||||
|
2014-06-01 4 7 7
|
||||||
|
2014-06-01 5 7 7
|
||||||
|
2014-06-01 6 7 7
|
||||||
|
2014-06-01 7 7 7
|
||||||
|
2014-06-01 8 7 7
|
||||||
|
2014-06-01 9 7 7
|
||||||
|
2014-06-01 7 7
|
||||||
|
0 333333 53 53
|
||||||
|
1 333333 53 53
|
||||||
|
2 333333 53 53
|
||||||
|
0 333333 53 53
|
||||||
|
1 333333 53 53
|
||||||
|
2 333333 53 53
|
211
tests/queries/0_stateless/01798_uniq_theta_sketch.sql
Normal file
211
tests/queries/0_stateless/01798_uniq_theta_sketch.sql
Normal file
@ -0,0 +1,211 @@
|
|||||||
|
SELECT 'uniqThetaSketch many agrs';
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
uniqThetaSketch(x), uniqThetaSketch((x)), uniqThetaSketch(x, y), uniqThetaSketch((x, y)), uniqThetaSketch(x, y, z), uniqThetaSketch((x, y, z))
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
number % 10 AS x,
|
||||||
|
intDiv(number, 10) % 10 AS y,
|
||||||
|
toString(intDiv(number, 100) % 10) AS z
|
||||||
|
FROM system.numbers LIMIT 1000
|
||||||
|
);
|
||||||
|
|
||||||
|
|
||||||
|
SELECT k,
|
||||||
|
uniqThetaSketch(x), uniqThetaSketch((x)), uniqThetaSketch(x, y), uniqThetaSketch((x, y)), uniqThetaSketch(x, y, z), uniqThetaSketch((x, y, z)),
|
||||||
|
count() AS c
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
(number + 0x8ffcbd8257219a26) * 0x66bb3430c06d2353 % 131 AS k,
|
||||||
|
number % 10 AS x,
|
||||||
|
intDiv(number, 10) % 10 AS y,
|
||||||
|
toString(intDiv(number, 100) % 10) AS z
|
||||||
|
FROM system.numbers LIMIT 100000
|
||||||
|
)
|
||||||
|
GROUP BY k
|
||||||
|
ORDER BY c DESC, k ASC
|
||||||
|
LIMIT 10;
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'uniqThetaSketch distinct';
|
||||||
|
|
||||||
|
SET count_distinct_implementation = 'uniqThetaSketch';
|
||||||
|
SELECT count(DISTINCT x) FROM (SELECT number % 123 AS x FROM system.numbers LIMIT 1000);
|
||||||
|
SELECT count(DISTINCT x, y) FROM (SELECT number % 11 AS x, number % 13 AS y FROM system.numbers LIMIT 1000);
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'uniqThetaSketch arrays';
|
||||||
|
|
||||||
|
SELECT uniqThetaSketchArray([0, 1, 1], [0, 1, 1], [0, 1, 1]);
|
||||||
|
SELECT uniqThetaSketchArray([0, 1, 1], [0, 1, 1], [0, 1, 0]);
|
||||||
|
SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[1, 2], [1, 2], [1, 2, 3], []]) AS x);
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'uniqThetaSketch complex types';
|
||||||
|
|
||||||
|
SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x);
|
||||||
|
SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[[]], [['a', 'b']], [['a'], ['b']], [['a', 'b']]]) AS x);
|
||||||
|
SELECT uniqThetaSketch(x, x) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x);
|
||||||
|
SELECT uniqThetaSketch(x, arrayMap(elem -> [elem, elem], x)) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x);
|
||||||
|
SELECT uniqThetaSketch(x, toString(x)) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x);
|
||||||
|
SELECT uniqThetaSketch((x, x)) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x);
|
||||||
|
SELECT uniqThetaSketch((x, arrayMap(elem -> [elem, elem], x))) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x);
|
||||||
|
SELECT uniqThetaSketch((x, toString(x))) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x);
|
||||||
|
SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[], ['a'], ['a', NULL, 'b'], []]) AS x);
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'uniqThetaSketch decimals';
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS decimal;
|
||||||
|
CREATE TABLE decimal
|
||||||
|
(
|
||||||
|
a Decimal32(4),
|
||||||
|
b Decimal64(8),
|
||||||
|
c Decimal128(8)
|
||||||
|
) ENGINE = Memory;
|
||||||
|
|
||||||
|
SELECT (uniqThetaSketch(a), uniqThetaSketch(b), uniqThetaSketch(c))
|
||||||
|
FROM (SELECT * FROM decimal ORDER BY a);
|
||||||
|
|
||||||
|
INSERT INTO decimal (a, b, c)
|
||||||
|
SELECT toDecimal32(number - 50, 4), toDecimal64(number - 50, 8) / 3, toDecimal128(number - 50, 8) / 5
|
||||||
|
FROM system.numbers LIMIT 101;
|
||||||
|
|
||||||
|
SELECT (uniqThetaSketch(a), uniqThetaSketch(b), uniqThetaSketch(c))
|
||||||
|
FROM (SELECT * FROM decimal ORDER BY a);
|
||||||
|
|
||||||
|
DROP TABLE decimal;
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'uniqThetaSketch remove injective';
|
||||||
|
|
||||||
|
set optimize_injective_functions_inside_uniq = 1;
|
||||||
|
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(x) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(x + y) from (select number % 2 as x, number % 3 y from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(-x) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(bitNot(x)) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(bitNot(-x)) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(-bitNot(-x)) from (select number % 2 as x from numbers(10));
|
||||||
|
|
||||||
|
set optimize_injective_functions_inside_uniq = 0;
|
||||||
|
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(x) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(x + y) from (select number % 2 as x, number % 3 y from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(-x) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(bitNot(x)) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(bitNot(-x)) from (select number % 2 as x from numbers(10));
|
||||||
|
EXPLAIN SYNTAX select uniqThetaSketch(-bitNot(-x)) from (select number % 2 as x from numbers(10));
|
||||||
|
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS stored_aggregates;
|
||||||
|
|
||||||
|
-- simple
|
||||||
|
CREATE TABLE stored_aggregates
|
||||||
|
(
|
||||||
|
d Date,
|
||||||
|
Uniq AggregateFunction(uniq, UInt64),
|
||||||
|
UniqThetaSketch AggregateFunction(uniqThetaSketch, UInt64)
|
||||||
|
)
|
||||||
|
ENGINE = AggregatingMergeTree(d, d, 8192);
|
||||||
|
|
||||||
|
INSERT INTO stored_aggregates
|
||||||
|
SELECT
|
||||||
|
toDate('2014-06-01') AS d,
|
||||||
|
uniqState(number) AS Uniq,
|
||||||
|
uniqThetaSketchState(number) AS UniqThetaSketch
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT * FROM system.numbers LIMIT 1000
|
||||||
|
);
|
||||||
|
|
||||||
|
SELECT uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates;
|
||||||
|
|
||||||
|
SELECT d, uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates GROUP BY d ORDER BY d;
|
||||||
|
|
||||||
|
OPTIMIZE TABLE stored_aggregates;
|
||||||
|
|
||||||
|
SELECT uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates;
|
||||||
|
|
||||||
|
SELECT d, uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates GROUP BY d ORDER BY d;
|
||||||
|
|
||||||
|
DROP TABLE stored_aggregates;
|
||||||
|
|
||||||
|
-- complex
|
||||||
|
CREATE TABLE stored_aggregates
|
||||||
|
(
|
||||||
|
d Date,
|
||||||
|
k1 UInt64,
|
||||||
|
k2 String,
|
||||||
|
Uniq AggregateFunction(uniq, UInt64),
|
||||||
|
UniqThetaSketch AggregateFunction(uniqThetaSketch, UInt64)
|
||||||
|
)
|
||||||
|
ENGINE = AggregatingMergeTree(d, (d, k1, k2), 8192);
|
||||||
|
|
||||||
|
INSERT INTO stored_aggregates
|
||||||
|
SELECT
|
||||||
|
toDate('2014-06-01') AS d,
|
||||||
|
intDiv(number, 100) AS k1,
|
||||||
|
toString(intDiv(number, 10)) AS k2,
|
||||||
|
uniqState(toUInt64(number % 7)) AS Uniq,
|
||||||
|
uniqThetaSketchState(toUInt64(number % 7)) AS UniqThetaSketch
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT * FROM system.numbers LIMIT 1000
|
||||||
|
)
|
||||||
|
GROUP BY d, k1, k2
|
||||||
|
ORDER BY d, k1, k2;
|
||||||
|
|
||||||
|
SELECT d, k1, k2,
|
||||||
|
uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch)
|
||||||
|
FROM stored_aggregates
|
||||||
|
GROUP BY d, k1, k2
|
||||||
|
ORDER BY d, k1, k2;
|
||||||
|
|
||||||
|
SELECT d, k1,
|
||||||
|
uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch)
|
||||||
|
FROM stored_aggregates
|
||||||
|
GROUP BY d, k1
|
||||||
|
ORDER BY d, k1;
|
||||||
|
|
||||||
|
SELECT d,
|
||||||
|
uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch)
|
||||||
|
FROM stored_aggregates
|
||||||
|
GROUP BY d
|
||||||
|
ORDER BY d;
|
||||||
|
|
||||||
|
DROP TABLE stored_aggregates;
|
||||||
|
|
||||||
|
---- sum + uniq with more data
|
||||||
|
drop table if exists summing_merge_tree_null;
|
||||||
|
drop table if exists summing_merge_tree_aggregate_function;
|
||||||
|
create table summing_merge_tree_null (
|
||||||
|
d materialized today(),
|
||||||
|
k UInt64,
|
||||||
|
c UInt64,
|
||||||
|
u UInt64
|
||||||
|
) engine=Null;
|
||||||
|
|
||||||
|
create materialized view summing_merge_tree_aggregate_function (
|
||||||
|
d Date,
|
||||||
|
k UInt64,
|
||||||
|
c UInt64,
|
||||||
|
un AggregateFunction(uniq, UInt64),
|
||||||
|
ut AggregateFunction(uniqThetaSketch, UInt64)
|
||||||
|
) engine=SummingMergeTree(d, k, 8192)
|
||||||
|
as select d, k, sum(c) as c, uniqState(u) as un, uniqThetaSketchState(u) as ut
|
||||||
|
from summing_merge_tree_null
|
||||||
|
group by d, k;
|
||||||
|
|
||||||
|
-- prime number 53 to avoid resonanse between %3 and %53
|
||||||
|
insert into summing_merge_tree_null select number % 3, 1, number % 53 from numbers(999999);
|
||||||
|
|
||||||
|
select k, sum(c), uniqMerge(un), uniqThetaSketchMerge(ut) from summing_merge_tree_aggregate_function group by k order by k;
|
||||||
|
optimize table summing_merge_tree_aggregate_function;
|
||||||
|
select k, sum(c), uniqMerge(un), uniqThetaSketchMerge(ut) from summing_merge_tree_aggregate_function group by k order by k;
|
||||||
|
|
||||||
|
drop table summing_merge_tree_aggregate_function;
|
||||||
|
drop table summing_merge_tree_null;
|
||||||
|
|
219
tests/queries/0_stateless/01799_long_uniq_theta_sketch.reference
Normal file
219
tests/queries/0_stateless/01799_long_uniq_theta_sketch.reference
Normal file
@ -0,0 +1,219 @@
|
|||||||
|
uniqThetaSketch
|
||||||
|
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 55018
|
||||||
|
1 54020
|
||||||
|
3 53774
|
||||||
|
6 53947
|
||||||
|
7 53839
|
||||||
|
9 54408
|
||||||
|
10 26876
|
||||||
|
11 54985
|
||||||
|
13 53479
|
||||||
|
14 53516
|
||||||
|
17 53331
|
||||||
|
19 53680
|
||||||
|
20 54211
|
||||||
|
21 53054
|
||||||
|
22 54690
|
||||||
|
26 53716
|
||||||
|
31 54139
|
||||||
|
35 52331
|
||||||
|
36 53766
|
||||||
|
uniqThetaSketch round(float)
|
||||||
|
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 53054
|
||||||
|
0.125 53839
|
||||||
|
0.5 54020
|
||||||
|
0.05 53680
|
||||||
|
0.143 53947
|
||||||
|
0.091 26876
|
||||||
|
0.056 53331
|
||||||
|
0.048 54211
|
||||||
|
0.083 54985
|
||||||
|
0.25 53774
|
||||||
|
1 55018
|
||||||
|
0.1 54408
|
||||||
|
0.028 52331
|
||||||
|
0.027 53766
|
||||||
|
0.031 54139
|
||||||
|
0.067 53516
|
||||||
|
0.043 54690
|
||||||
|
0.037 53716
|
||||||
|
0.071 53479
|
||||||
|
uniqThetaSketch round(toFloat32())
|
||||||
|
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 54020
|
||||||
|
0.05 53680
|
||||||
|
0.25 53774
|
||||||
|
0.048 54211
|
||||||
|
0.091 26876
|
||||||
|
0.043 54690
|
||||||
|
0.071 53479
|
||||||
|
0.083 54985
|
||||||
|
0.125 53839
|
||||||
|
0.031 54139
|
||||||
|
0.143 53947
|
||||||
|
0.028 52331
|
||||||
|
0.067 53516
|
||||||
|
0.045 53054
|
||||||
|
0.027 53766
|
||||||
|
0.056 53331
|
||||||
|
0.037 53716
|
||||||
|
0.1 54408
|
||||||
|
1 55018
|
||||||
|
uniqThetaSketch IPv4NumToString
|
||||||
|
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 54929
|
||||||
|
1 53802
|
||||||
|
3 54706
|
||||||
|
6 54700
|
||||||
|
7 53592
|
||||||
|
9 54036
|
||||||
|
10 27392
|
||||||
|
11 53768
|
||||||
|
13 54566
|
||||||
|
14 53104
|
||||||
|
17 54243
|
||||||
|
19 55003
|
||||||
|
20 53398
|
||||||
|
21 53831
|
||||||
|
22 54603
|
||||||
|
26 54607
|
||||||
|
31 54012
|
||||||
|
35 54826
|
||||||
|
36 54910
|
||||||
|
uniqThetaSketch remote()
|
||||||
|
1
|
||||||
|
uniqThetaSketch precise
|
||||||
|
10000000
|
||||||
|
10021957
|
||||||
|
10021969
|
||||||
|
10094819
|
35
tests/queries/0_stateless/01799_long_uniq_theta_sketch.sql
Normal file
35
tests/queries/0_stateless/01799_long_uniq_theta_sketch.sql
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
SELECT 'uniqThetaSketch';
|
||||||
|
|
||||||
|
SELECT Y, uniqThetaSketch(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, uniqThetaSketch(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, uniqThetaSketch(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||||
|
|
||||||
|
SELECT 'uniqThetaSketch round(float)';
|
||||||
|
|
||||||
|
SELECT Y, uniqThetaSketch(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, uniqThetaSketch(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, uniqThetaSketch(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 'uniqThetaSketch round(toFloat32())';
|
||||||
|
|
||||||
|
SELECT Y, uniqThetaSketch(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, uniqThetaSketch(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, uniqThetaSketch(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 'uniqThetaSketch IPv4NumToString';
|
||||||
|
|
||||||
|
SELECT Y, uniqThetaSketch(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, uniqThetaSketch(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, uniqThetaSketch(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 'uniqThetaSketch remote()';
|
||||||
|
|
||||||
|
SELECT uniqThetaSketch(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'uniqThetaSketch precise';
|
||||||
|
SELECT uniqExact(number) FROM numbers(1e7);
|
||||||
|
SELECT uniqCombined(number) FROM numbers(1e7);
|
||||||
|
SELECT uniqCombined64(number) FROM numbers(1e7);
|
||||||
|
SELECT uniqThetaSketch(number) FROM numbers(1e7);
|
||||||
|
|
@ -228,6 +228,8 @@
|
|||||||
01780_clickhouse_dictionary_source_loop
|
01780_clickhouse_dictionary_source_loop
|
||||||
01790_dist_INSERT_block_structure_mismatch_types_and_names
|
01790_dist_INSERT_block_structure_mismatch_types_and_names
|
||||||
01791_dist_INSERT_block_structure_mismatch
|
01791_dist_INSERT_block_structure_mismatch
|
||||||
|
01798_uniq_theta_sketch
|
||||||
|
01799_long_uniq_theta_sketch
|
||||||
01801_distinct_group_by_shard
|
01801_distinct_group_by_shard
|
||||||
01804_dictionary_decimal256_type
|
01804_dictionary_decimal256_type
|
||||||
01801_s3_distributed
|
01801_s3_distributed
|
||||||
|
Loading…
Reference in New Issue
Block a user