Merge pull request #67013 from JackyWoo/add_statistics_minmax

Add `min_max` statistics type
This commit is contained in:
Robert Schulze 2024-09-06 16:14:04 +00:00 committed by GitHub
commit a73eb1c177
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
13 changed files with 307 additions and 14 deletions

View File

@ -989,7 +989,11 @@ ALTER TABLE tab DROP STATISTICS a;
These lightweight statistics aggregate information about distribution of values in columns. Statistics are stored in every part and updated when every insert comes.
They can be used for prewhere optimization only if we enable `set allow_statistics_optimize = 1`.
#### Available Types of Column Statistics {#available-types-of-column-statistics}
### Available Types of Column Statistics {#available-types-of-column-statistics}
- `MinMax`
The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns.
- `TDigest`
@ -1003,6 +1007,27 @@ They can be used for prewhere optimization only if we enable `set allow_statisti
[Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column.
### Supported Data Types {#supported-data-types}
| | (U)Int* | Float* | Decimal(*) | Date* | Boolean | Enum* | (Fixed)String |
|-----------|---------|--------|------------|-------|---------|-------|------------------|
| count_min | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| MinMax | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✗ |
| TDigest | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✗ |
| Uniq | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
### Supported Operations {#supported-operations}
| | Equality filters (==) | Range filters (>, >=, <, <=) |
|-----------|-----------------------|------------------------------|
| count_min | ✔ | ✗ |
| MinMax | ✗ | ✔ |
| TDigest | ✗ | ✔ |
| Uniq | ✔ | ✗ |
## Column-level Settings {#column-level-settings}
Certain MergeTree settings can be overridden at column level:

View File

@ -9,6 +9,7 @@
#include <Storages/ColumnsDescription.h>
#include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <Storages/Statistics/StatisticsCountMinSketch.h>
#include <Storages/Statistics/StatisticsMinMax.h>
#include <Storages/Statistics/StatisticsTDigest.h>
#include <Storages/Statistics/StatisticsUniq.h>
#include <Storages/StatisticsDescription.h>
@ -101,6 +102,8 @@ Float64 ColumnStatistics::estimateLess(const Field & val) const
{
if (stats.contains(StatisticsType::TDigest))
return stats.at(StatisticsType::TDigest)->estimateLess(val);
if (stats.contains(StatisticsType::MinMax))
return stats.at(StatisticsType::MinMax)->estimateLess(val);
return rows * ConditionSelectivityEstimator::default_cond_range_factor;
}
@ -121,6 +124,14 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const
if (stats.contains(StatisticsType::CountMinSketch))
return stats.at(StatisticsType::CountMinSketch)->estimateEqual(val);
#endif
if (stats.contains(StatisticsType::Uniq))
{
UInt64 cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality();
if (cardinality == 0 || rows == 0)
return 0;
return 1.0 / cardinality * rows; /// assume uniform distribution
}
return rows * ConditionSelectivityEstimator::default_cond_equal_factor;
}
@ -198,6 +209,9 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va
MergeTreeStatisticsFactory::MergeTreeStatisticsFactory()
{
registerValidator(StatisticsType::MinMax, minMaxStatisticsValidator);
registerCreator(StatisticsType::MinMax, minMaxStatisticsCreator);
registerValidator(StatisticsType::TDigest, tdigestStatisticsValidator);
registerCreator(StatisticsType::TDigest, tdigestStatisticsCreator);
@ -234,7 +248,7 @@ ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnDescription & co
{
auto it = creators.find(type);
if (it == creators.end())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type);
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'count_min', 'minmax', 'tdigest' and 'uniq'", type);
auto stat_ptr = (it->second)(desc, column_desc.type);
column_stat->stats[type] = stat_ptr;
}

View File

@ -1,4 +1,3 @@
#include <Storages/Statistics/StatisticsCountMinSketch.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>

View File

@ -0,0 +1,86 @@
#include <Storages/Statistics/StatisticsMinMax.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <algorithm>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_STATISTICS;
}
StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & description, const DataTypePtr & data_type_)
: IStatistics(description)
, data_type(data_type_)
{
}
void StatisticsMinMax::update(const ColumnPtr & column)
{
for (size_t row = 0; row < column->size(); ++row)
{
if (column->isNullAt(row))
continue;
auto value = column->getFloat64(row);
min = std::min(value, min);
max = std::max(value, max);
}
row_count += column->size();
}
void StatisticsMinMax::serialize(WriteBuffer & buf)
{
writeIntBinary(row_count, buf);
writeFloatBinary(min, buf);
writeFloatBinary(max, buf);
}
void StatisticsMinMax::deserialize(ReadBuffer & buf)
{
readIntBinary(row_count, buf);
readFloatBinary(min, buf);
readFloatBinary(max, buf);
}
Float64 StatisticsMinMax::estimateLess(const Field & val) const
{
if (row_count == 0)
return 0;
auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type);
if (!val_as_float.has_value())
return 0;
if (val_as_float < min)
return 0;
if (val_as_float > max)
return row_count;
if (min == max)
return (val_as_float != max) ? 0 : row_count;
return ((*val_as_float - min) / (max - min)) * row_count;
}
void minMaxStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type)
{
auto inner_data_type = removeNullable(data_type);
inner_data_type = removeLowCardinalityAndNullable(inner_data_type);
if (!inner_data_type->isValueRepresentedByNumber())
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", data_type->getName());
}
StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
{
return std::make_shared<StatisticsMinMax>(description, data_type);
}
}

View File

@ -0,0 +1,33 @@
#pragma once
#include <Storages/Statistics/Statistics.h>
#include <DataTypes/IDataType.h>
namespace DB
{
class StatisticsMinMax : public IStatistics
{
public:
StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_);
void update(const ColumnPtr & column) override;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
Float64 estimateLess(const Field & val) const override;
private:
Float64 min = std::numeric_limits<Float64>::max();
Float64 max = std::numeric_limits<Float64>::min();
UInt64 row_count = 0;
DataTypePtr data_type;
};
void minMaxStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
}

View File

@ -56,7 +56,7 @@ void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/
{
DataTypePtr inner_data_type = removeNullable(data_type);
inner_data_type = removeLowCardinalityAndNullable(inner_data_type);
if (!inner_data_type->isValueRepresentedByNumber())
if (!inner_data_type->isValueRepresentedByNumber() && !isStringOrFixedString(inner_data_type))
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName());
}

View File

@ -50,7 +50,9 @@ static StatisticsType stringToStatisticsType(String type)
return StatisticsType::Uniq;
if (type == "count_min")
return StatisticsType::CountMinSketch;
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq' and 'count_min'.", type);
if (type == "minmax")
return StatisticsType::MinMax;
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type);
}
String SingleStatisticsDescription::getTypeName() const
@ -63,8 +65,10 @@ String SingleStatisticsDescription::getTypeName() const
return "Uniq";
case StatisticsType::CountMinSketch:
return "count_min";
case StatisticsType::MinMax:
return "minmax";
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq' and 'count_min'.", type);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type);
}
}

View File

@ -14,6 +14,7 @@ enum class StatisticsType : UInt8
TDigest = 0,
Uniq = 1,
CountMinSketch = 2,
MinMax = 3,
Max = 63,
};

View File

@ -0,0 +1,5 @@
Test create statistics:
CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(tdigest, uniq, count_min, minmax),\n `b` LowCardinality(Nullable(String)) STATISTICS(uniq, count_min),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(tdigest, uniq, count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192
Test materialize and drop statistics:
CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` LowCardinality(Nullable(String)) STATISTICS(uniq, count_min),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192
CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` LowCardinality(Nullable(String)),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,35 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS tab SYNC;
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
SET allow_suspicious_low_cardinality_types=1;
SET mutations_sync = 2;
SELECT 'Test create statistics:';
CREATE TABLE tab
(
a LowCardinality(Int64) STATISTICS(count_min, minmax, tdigest, uniq),
b LowCardinality(Nullable(String)) STATISTICS(count_min, uniq),
c LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax, tdigest, uniq),
d DateTime STATISTICS(count_min, minmax, tdigest, uniq),
pk String,
) Engine = MergeTree() ORDER BY pk;
INSERT INTO tab select number, number, number, toDateTime(number), generateUUIDv4() FROM system.numbers LIMIT 10000;
SHOW CREATE TABLE tab;
SELECT 'Test materialize and drop statistics:';
ALTER TABLE tab DROP STATISTICS a, b, c, d;
ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq;
ALTER TABLE tab MATERIALIZE STATISTICS b;
SHOW CREATE TABLE tab;
ALTER TABLE tab DROP STATISTICS b;
SHOW CREATE TABLE tab;
DROP TABLE IF EXISTS tab SYNC;

View File

@ -7,6 +7,7 @@ SET mutations_sync = 1;
DROP TABLE IF EXISTS tab;
SET allow_experimental_statistics = 0;
-- Error case: Can't create statistics when allow_experimental_statistics = 0
CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY }
@ -46,7 +47,7 @@ CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTre
CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
-- uniq requires data_type.isValueRepresentedByInteger
-- uniq requires data_type.isValueRepresentedByInteger or (Fixed)String
-- These types work:
CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
@ -61,9 +62,9 @@ CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple
CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
-- These types don't work:
CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
@ -94,6 +95,30 @@ CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeT
CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
-- minmax requires data_type.isValueRepresentedByInteger
-- These types work:
CREATE TABLE tab (col UInt8 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col UInt256 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Float32 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Decimal32(3) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date32 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime64 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col IPv4 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Nullable(UInt8) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
-- These types don't work:
CREATE TABLE tab (col String STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col FixedString(1) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Array(Float64) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col UUID STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col IPv6 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
-- CREATE TABLE was easy, ALTER is more fun
CREATE TABLE tab
@ -173,6 +198,13 @@ ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATI
-- Doesn't work:
ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS }
-- minmax
-- Works:
ALTER TABLE tab ADD STATISTICS f64 TYPE minmax; ALTER TABLE tab DROP STATISTICS f64;
ALTER TABLE tab MODIFY STATISTICS f64 TYPE minmax; ALTER TABLE tab DROP STATISTICS f64;
-- Doesn't work:
ALTER TABLE tab ADD STATISTICS a TYPE minmax; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MODIFY STATISTICS a TYPE minmax; -- { serverError ILLEGAL_STATISTICS }
-- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing
-- statistics objects (e.g. tdigest can be created on Float64 and UInt64)

View File

@ -3,10 +3,13 @@ u64 and =
10
10
10
10
0
0
0
0
0
10
10
10
10
@ -16,10 +19,13 @@ u64 and <
70
70
70
70
80
80
80
80
80
70
70
70
70
@ -29,6 +35,8 @@ f64 and =
10
10
10
10
0
0
0
0
@ -37,6 +45,8 @@ f64 and =
10
10
10
10
0
0
0
0
@ -46,6 +56,8 @@ f64 and <
70
70
70
70
80
80
80
80
@ -54,6 +66,8 @@ f64 and <
70
70
70
70
80
80
80
80
@ -63,6 +77,8 @@ dt and =
0
0
0
0
10
10
10
10
@ -72,6 +88,8 @@ dt and <
10000
10000
10000
10000
70
70
70
70
@ -89,6 +107,10 @@ b and =
5000
5000
5000
5000
5000
5000
0
0
0
0
@ -96,3 +118,4 @@ b and =
s and =
10
10
10

View File

@ -12,46 +12,56 @@ CREATE TABLE tab
(
u64 UInt64,
u64_tdigest UInt64 STATISTICS(tdigest),
u64_minmax UInt64 STATISTICS(minmax),
u64_count_min UInt64 STATISTICS(count_min),
u64_uniq UInt64 STATISTICS(uniq),
f64 Float64,
f64_tdigest Float64 STATISTICS(tdigest),
f64_minmax Float64 STATISTICS(minmax),
f64_count_min Float64 STATISTICS(count_min),
f64_uniq Float64 STATISTICS(uniq),
dt DateTime,
dt_tdigest DateTime STATISTICS(tdigest),
dt_minmax DateTime STATISTICS(minmax),
dt_count_min DateTime STATISTICS(count_min),
dt_uniq DateTime STATISTICS(uniq),
b Bool,
b_tdigest Bool STATISTICS(tdigest),
b_minmax Bool STATISTICS(minmax),
b_count_min Bool STATISTICS(count_min),
b_uniq Bool STATISTICS(uniq),
s String,
-- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest
s_count_min String STATISTICS(count_min)
-- s_uniq String STATISTICS(uniq), -- not supported by uniq
-- s_minmax String STATISTICS(minmax), -- not supported by minmax
s_count_min String STATISTICS(count_min),
s_uniq String STATISTICS(uniq)
) Engine = MergeTree() ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO tab
-- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000;
SELECT number % 1000,
SELECT number % 1000, -- u64
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000, -- f64
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000, -- dt
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 2, -- b
number % 2,
number % 2,
number % 2,
number % 2,
toString(number % 1000),
toString(number % 1000),
toString(number % 1000)
FROM system.numbers LIMIT 10000;
@ -61,21 +71,25 @@ SELECT 'u64 and =';
SELECT count(*) FROM tab WHERE u64 = 7;
SELECT count(*) FROM tab WHERE u64_tdigest = 7;
SELECT count(*) FROM tab WHERE u64_minmax = 7;
SELECT count(*) FROM tab WHERE u64_count_min = 7;
SELECT count(*) FROM tab WHERE u64_uniq = 7;
SELECT count(*) FROM tab WHERE u64 = 7.7;
SELECT count(*) FROM tab WHERE u64_tdigest = 7.7;
SELECT count(*) FROM tab WHERE u64_minmax = 7.7;
SELECT count(*) FROM tab WHERE u64_count_min = 7.7;
SELECT count(*) FROM tab WHERE u64_uniq = 7.7;
SELECT count(*) FROM tab WHERE u64 = '7';
SELECT count(*) FROM tab WHERE u64_tdigest = '7';
SELECT count(*) FROM tab WHERE u64_minmax = '7';
SELECT count(*) FROM tab WHERE u64_count_min = '7';
SELECT count(*) FROM tab WHERE u64_uniq = '7';
SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_minmax = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH }
@ -83,21 +97,25 @@ SELECT 'u64 and <';
SELECT count(*) FROM tab WHERE u64 < 7;
SELECT count(*) FROM tab WHERE u64_tdigest < 7;
SELECT count(*) FROM tab WHERE u64_minmax < 7;
SELECT count(*) FROM tab WHERE u64_count_min < 7;
SELECT count(*) FROM tab WHERE u64_uniq < 7;
SELECT count(*) FROM tab WHERE u64 < 7.7;
SELECT count(*) FROM tab WHERE u64_tdigest < 7.7;
SELECT count(*) FROM tab WHERE u64_minmax < 7.7;
SELECT count(*) FROM tab WHERE u64_count_min < 7.7;
SELECT count(*) FROM tab WHERE u64_uniq < 7.7;
SELECT count(*) FROM tab WHERE u64 < '7';
SELECT count(*) FROM tab WHERE u64_tdigest < '7';
SELECT count(*) FROM tab WHERE u64_minmax < '7';
SELECT count(*) FROM tab WHERE u64_count_min < '7';
SELECT count(*) FROM tab WHERE u64_uniq < '7';
SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_minmax < '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH }
@ -107,21 +125,25 @@ SELECT 'f64 and =';
SELECT count(*) FROM tab WHERE f64 = 7;
SELECT count(*) FROM tab WHERE f64_tdigest = 7;
SELECT count(*) FROM tab WHERE f64_minmax = 7;
SELECT count(*) FROM tab WHERE f64_count_min = 7;
SELECT count(*) FROM tab WHERE f64_uniq = 7;
SELECT count(*) FROM tab WHERE f64 = 7.7;
SELECT count(*) FROM tab WHERE f64_tdigest = 7.7;
SELECT count(*) FROM tab WHERE f64_minmax = 7.7;
SELECT count(*) FROM tab WHERE f64_count_min = 7.7;
SELECT count(*) FROM tab WHERE f64_uniq = 7.7;
SELECT count(*) FROM tab WHERE f64 = '7';
SELECT count(*) FROM tab WHERE f64_tdigest = '7';
SELECT count(*) FROM tab WHERE f64_minmax = '7';
SELECT count(*) FROM tab WHERE f64_count_min = '7';
SELECT count(*) FROM tab WHERE f64_uniq = '7';
SELECT count(*) FROM tab WHERE f64 = '7.7';
SELECT count(*) FROM tab WHERE f64_tdigest = '7.7';
SELECT count(*) FROM tab WHERE f64_minmax = '7.7';
SELECT count(*) FROM tab WHERE f64_count_min = '7.7';
SELECT count(*) FROM tab WHERE f64_uniq = '7.7';
@ -129,21 +151,25 @@ SELECT 'f64 and <';
SELECT count(*) FROM tab WHERE f64 < 7;
SELECT count(*) FROM tab WHERE f64_tdigest < 7;
SELECT count(*) FROM tab WHERE f64_minmax < 7;
SELECT count(*) FROM tab WHERE f64_count_min < 7;
SELECT count(*) FROM tab WHERE f64_uniq < 7;
SELECT count(*) FROM tab WHERE f64 < 7.7;
SELECT count(*) FROM tab WHERE f64_tdigest < 7.7;
SELECT count(*) FROM tab WHERE f64_minmax < 7.7;
SELECT count(*) FROM tab WHERE f64_count_min < 7.7;
SELECT count(*) FROM tab WHERE f64_uniq < 7.7;
SELECT count(*) FROM tab WHERE f64 < '7';
SELECT count(*) FROM tab WHERE f64_tdigest < '7';
SELECT count(*) FROM tab WHERE f64_minmax < '7';
SELECT count(*) FROM tab WHERE f64_count_min < '7';
SELECT count(*) FROM tab WHERE f64_uniq < '7';
SELECT count(*) FROM tab WHERE f64 < '7.7';
SELECT count(*) FROM tab WHERE f64_tdigest < '7.7';
SELECT count(*) FROM tab WHERE f64_minmax < '7.7';
SELECT count(*) FROM tab WHERE f64_count_min < '7.7';
SELECT count(*) FROM tab WHERE f64_uniq < '7.7';
@ -153,11 +179,13 @@ SELECT 'dt and =';
SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_minmax = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt = 7;
SELECT count(*) FROM tab WHERE dt_tdigest = 7;
SELECT count(*) FROM tab WHERE dt_minmax = 7;
SELECT count(*) FROM tab WHERE dt_count_min = 7;
SELECT count(*) FROM tab WHERE dt_uniq = 7;
@ -165,11 +193,13 @@ SELECT 'dt and <';
SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_minmax < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt < 7;
SELECT count(*) FROM tab WHERE dt_tdigest < 7;
SELECT count(*) FROM tab WHERE dt_minmax < 7;
SELECT count(*) FROM tab WHERE dt_count_min < 7;
SELECT count(*) FROM tab WHERE dt_uniq < 7;
@ -179,21 +209,25 @@ SELECT 'b and =';
SELECT count(*) FROM tab WHERE b = true;
SELECT count(*) FROM tab WHERE b_tdigest = true;
SELECT count(*) FROM tab WHERE b_minmax = true;
SELECT count(*) FROM tab WHERE b_count_min = true;
SELECT count(*) FROM tab WHERE b_uniq = true;
SELECT count(*) FROM tab WHERE b = 'true';
SELECT count(*) FROM tab WHERE b_tdigest = 'true';
SELECT count(*) FROM tab WHERE b_minmax = 'true';
SELECT count(*) FROM tab WHERE b_count_min = 'true';
SELECT count(*) FROM tab WHERE b_uniq = 'true';
SELECT count(*) FROM tab WHERE b = 1;
SELECT count(*) FROM tab WHERE b_tdigest = 1;
SELECT count(*) FROM tab WHERE b_minmax = 1;
SELECT count(*) FROM tab WHERE b_count_min = 1;
SELECT count(*) FROM tab WHERE b_uniq = 1;
SELECT count(*) FROM tab WHERE b = 1.1;
SELECT count(*) FROM tab WHERE b_tdigest = 1.1;
SELECT count(*) FROM tab WHERE b_minmax = 1.1;
SELECT count(*) FROM tab WHERE b_count_min = 1.1;
SELECT count(*) FROM tab WHERE b_uniq = 1.1;
@ -203,12 +237,14 @@ SELECT 's and =';
SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE }
-- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported
-- SELECT count(*) FROM tab WHERE s_minmax = 7; -- not supported
SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE }
-- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported
SELECT count(*) FROM tab WHERE s_uniq = 7; -- { serverError NO_COMMON_TYPE }
SELECT count(*) FROM tab WHERE s = '7';
-- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported
-- SELECT count(*) FROM tab WHERE s_minmax = '7'; -- not supported
SELECT count(*) FROM tab WHERE s_count_min = '7';
-- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported
SELECT count(*) FROM tab WHERE s_uniq = '7';
DROP TABLE tab;