mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #67013 from JackyWoo/add_statistics_minmax
Add `min_max` statistics type
This commit is contained in:
commit
a73eb1c177
@ -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:
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -1,4 +1,3 @@
|
||||
|
||||
#include <Storages/Statistics/StatisticsCountMinSketch.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
86
src/Storages/Statistics/StatisticsMinMax.cpp
Normal file
86
src/Storages/Statistics/StatisticsMinMax.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
33
src/Storages/Statistics/StatisticsMinMax.h
Normal file
33
src/Storages/Statistics/StatisticsMinMax.h
Normal 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);
|
||||
|
||||
}
|
@ -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());
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -14,6 +14,7 @@ enum class StatisticsType : UInt8
|
||||
TDigest = 0,
|
||||
Uniq = 1,
|
||||
CountMinSketch = 2,
|
||||
MinMax = 3,
|
||||
|
||||
Max = 63,
|
||||
};
|
||||
|
@ -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
|
@ -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;
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user