Rename IStatistics to ISingleStatistics

This commit is contained in:
JackyWoo 2024-09-09 15:35:21 +08:00
parent 26f21a0d94
commit aa332b778a
12 changed files with 67 additions and 69 deletions

View File

@ -54,7 +54,7 @@ std::optional<Float64> StatisticsUtils::tryConvertToFloat64(const Field & value,
return {};
}
IStatistics::IStatistics(const SingleStatisticsDescription & stat_)
ISingleStatistics::ISingleStatistics(const SingleStatisticsDescription & stat_)
: stat(stat_)
{
}
@ -71,17 +71,17 @@ void ColumnStatistics::update(const ColumnPtr & column)
stat.second->update(column);
}
UInt64 IStatistics::estimateCardinality() const
UInt64 ISingleStatistics::estimateCardinality() const
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cardinality estimation is not implemented for this type of statistics");
}
Float64 IStatistics::estimateEqual(const Field & /*val*/) const
Float64 ISingleStatistics::estimateEqual(const Field & /*val*/) const
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Equality estimation is not implemented for this type of statistics");
}
Float64 IStatistics::estimateLess(const Field & /*val*/) const
Float64 ISingleStatistics::estimateLess(const Field & /*val*/) const
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Less-than estimation is not implemented for this type of statistics");
}
@ -100,10 +100,10 @@ Float64 IStatistics::estimateLess(const Field & /*val*/) const
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);
if (stats.contains(SingleStatisticsType::TDigest))
return stats.at(SingleStatisticsType::TDigest)->estimateLess(val);
if (stats.contains(SingleStatisticsType::MinMax))
return stats.at(SingleStatisticsType::MinMax)->estimateLess(val);
return rows * ConditionSelectivityEstimator::default_cond_range_factor;
}
@ -114,19 +114,19 @@ Float64 ColumnStatistics::estimateGreater(const Field & val) const
Float64 ColumnStatistics::estimateEqual(const Field & val) const
{
if (stats_desc.data_type->isValueRepresentedByNumber() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest))
if (stats_desc.data_type->isValueRepresentedByNumber() && stats.contains(SingleStatisticsType::Uniq) && stats.contains(SingleStatisticsType::TDigest))
{
/// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows) for every bucket.
if (stats.at(StatisticsType::Uniq)->estimateCardinality() < 2048)
return stats.at(StatisticsType::TDigest)->estimateEqual(val);
if (stats.at(SingleStatisticsType::Uniq)->estimateCardinality() < 2048)
return stats.at(SingleStatisticsType::TDigest)->estimateEqual(val);
}
#if USE_DATASKETCHES
if (stats.contains(StatisticsType::CountMinSketch))
return stats.at(StatisticsType::CountMinSketch)->estimateEqual(val);
if (stats.contains(SingleStatisticsType::CountMinSketch))
return stats.at(SingleStatisticsType::CountMinSketch)->estimateEqual(val);
#endif
if (stats.contains(StatisticsType::Uniq))
if (stats.contains(SingleStatisticsType::Uniq))
{
UInt64 cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality();
UInt64 cardinality = stats.at(SingleStatisticsType::Uniq)->estimateCardinality();
if (cardinality == 0 || rows == 0)
return 0;
return 1.0 / cardinality * rows; /// assume uniform distribution
@ -195,13 +195,13 @@ UInt64 ColumnStatistics::rowCount() const
return rows;
}
void MergeTreeStatisticsFactory::registerCreator(StatisticsType stats_type, Creator creator)
void MergeTreeStatisticsFactory::registerCreator(SingleStatisticsType stats_type, Creator creator)
{
if (!creators.emplace(stats_type, std::move(creator)).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics creator type {} is not unique", stats_type);
}
void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Validator validator)
void MergeTreeStatisticsFactory::registerValidator(SingleStatisticsType stats_type, Validator validator)
{
if (!validators.emplace(stats_type, std::move(validator)).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics validator type {} is not unique", stats_type);
@ -209,18 +209,18 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va
MergeTreeStatisticsFactory::MergeTreeStatisticsFactory()
{
registerValidator(StatisticsType::MinMax, minMaxStatisticsValidator);
registerCreator(StatisticsType::MinMax, minMaxStatisticsCreator);
registerValidator(SingleStatisticsType::MinMax, minMaxStatisticsValidator);
registerCreator(SingleStatisticsType::MinMax, minMaxStatisticsCreator);
registerValidator(StatisticsType::TDigest, tdigestStatisticsValidator);
registerCreator(StatisticsType::TDigest, tdigestStatisticsCreator);
registerValidator(SingleStatisticsType::TDigest, tdigestStatisticsValidator);
registerCreator(SingleStatisticsType::TDigest, tdigestStatisticsCreator);
registerValidator(StatisticsType::Uniq, uniqStatisticsValidator);
registerCreator(StatisticsType::Uniq, uniqStatisticsCreator);
registerValidator(SingleStatisticsType::Uniq, uniqStatisticsValidator);
registerCreator(SingleStatisticsType::Uniq, uniqStatisticsCreator);
#if USE_DATASKETCHES
registerValidator(StatisticsType::CountMinSketch, countMinSketchStatisticsValidator);
registerCreator(StatisticsType::CountMinSketch, countMinSketchStatisticsCreator);
registerValidator(SingleStatisticsType::CountMinSketch, countMinSketchStatisticsValidator);
registerCreator(SingleStatisticsType::CountMinSketch, countMinSketchStatisticsCreator);
#endif
}

View File

@ -2,8 +2,6 @@
#include <Core/Block.h>
#include <Core/Field.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
#include <Storages/StatisticsDescription.h>
#include <boost/core/noncopyable.hpp>
@ -21,15 +19,15 @@ struct StatisticsUtils
static std::optional<Float64> tryConvertToFloat64(const Field & value, const DataTypePtr & data_type);
};
/// Statistics describe properties of the values in the column,
/// SingleTypeStatistics describe properties of the values in the column,
/// e.g. how many unique values exist,
/// what are the N most frequent values,
/// how frequent is a value V, etc.
class IStatistics
class ISingleStatistics
{
public:
explicit IStatistics(const SingleStatisticsDescription & stat_);
virtual ~IStatistics() = default;
explicit ISingleStatistics(const SingleStatisticsDescription & stat_);
virtual ~ISingleStatistics() = default;
virtual void update(const ColumnPtr & column) = 0;
@ -49,7 +47,7 @@ protected:
SingleStatisticsDescription stat;
};
using StatisticsPtr = std::shared_ptr<IStatistics>;
using SingleStatisticsPtr = std::shared_ptr<ISingleStatistics>;
class ColumnStatistics
{
@ -74,7 +72,7 @@ private:
friend class MergeTreeStatisticsFactory;
ColumnStatisticsDescription stats_desc;
String column_name;
std::map<StatisticsType, StatisticsPtr> stats;
std::map<SingleStatisticsType, SingleStatisticsPtr> stats;
UInt64 rows = 0; /// the number of rows in the column
};
@ -91,20 +89,20 @@ public:
void validate(const ColumnStatisticsDescription & stats, const DataTypePtr & data_type) const;
using Validator = std::function<void(const SingleStatisticsDescription & stats, const DataTypePtr & data_type)>;
using Creator = std::function<StatisticsPtr(const SingleStatisticsDescription & stats, const DataTypePtr & data_type)>;
using Creator = std::function<SingleStatisticsPtr(const SingleStatisticsDescription & stats, const DataTypePtr & data_type)>;
ColumnStatisticsPtr get(const ColumnDescription & column_desc) const;
ColumnsStatistics getMany(const ColumnsDescription & columns) const;
void registerValidator(StatisticsType type, Validator validator);
void registerCreator(StatisticsType type, Creator creator);
void registerValidator(SingleStatisticsType type, Validator validator);
void registerCreator(SingleStatisticsType type, Creator creator);
protected:
MergeTreeStatisticsFactory();
private:
using Validators = std::unordered_map<StatisticsType, Validator>;
using Creators = std::unordered_map<StatisticsType, Creator>;
using Validators = std::unordered_map<SingleStatisticsType, Validator>;
using Creators = std::unordered_map<SingleStatisticsType, Creator>;
Validators validators;
Creators creators;
};

View File

@ -25,7 +25,7 @@ static constexpr auto num_hashes = 7uz;
static constexpr auto num_buckets = 2718uz;
StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & description, const DataTypePtr & data_type_)
: IStatistics(description)
: ISingleStatistics(description)
, sketch(num_hashes, num_buckets)
, data_type(data_type_)
{
@ -91,7 +91,7 @@ void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*des
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName());
}
StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
SingleStatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
{
return std::make_shared<StatisticsCountMinSketch>(description, data_type);
}

View File

@ -11,7 +11,7 @@
namespace DB
{
class StatisticsCountMinSketch : public IStatistics
class StatisticsCountMinSketch : public ISingleStatistics
{
public:
StatisticsCountMinSketch(const SingleStatisticsDescription & description, const DataTypePtr & data_type_);
@ -32,7 +32,7 @@ private:
void countMinSketchStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
SingleStatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
}

View File

@ -16,7 +16,7 @@ extern const int ILLEGAL_STATISTICS;
}
StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & description, const DataTypePtr & data_type_)
: IStatistics(description)
: ISingleStatistics(description)
, data_type(data_type_)
{
}
@ -78,7 +78,7 @@ void minMaxStatisticsValidator(const SingleStatisticsDescription & /*description
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)
SingleStatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
{
return std::make_shared<StatisticsMinMax>(description, data_type);
}

View File

@ -7,7 +7,7 @@
namespace DB
{
class StatisticsMinMax : public IStatistics
class StatisticsMinMax : public ISingleStatistics
{
public:
StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_);
@ -28,6 +28,6 @@ private:
};
void minMaxStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
SingleStatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
}

View File

@ -10,7 +10,7 @@ extern const int ILLEGAL_STATISTICS;
}
StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_)
: IStatistics(description)
: ISingleStatistics(description)
, data_type(data_type_)
{
}
@ -61,7 +61,7 @@ void tdigestStatisticsValidator(const SingleStatisticsDescription & /*descriptio
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName());
}
StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
SingleStatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
{
return std::make_shared<StatisticsTDigest>(description, data_type);
}

View File

@ -6,7 +6,7 @@
namespace DB
{
class StatisticsTDigest : public IStatistics
class StatisticsTDigest : public ISingleStatistics
{
public:
explicit StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_);
@ -25,6 +25,6 @@ private:
};
void tdigestStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
SingleStatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
}

View File

@ -12,7 +12,7 @@ namespace ErrorCodes
}
StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
: IStatistics(description)
: ISingleStatistics(description)
{
arena = std::make_unique<Arena>();
AggregateFunctionProperties properties;
@ -60,7 +60,7 @@ void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName());
}
StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
SingleStatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
{
return std::make_shared<StatisticsUniq>(description, data_type);
}

View File

@ -7,7 +7,7 @@
namespace DB
{
class StatisticsUniq : public IStatistics
class StatisticsUniq : public ISingleStatistics
{
public:
StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
@ -28,6 +28,6 @@ private:
};
void uniqStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
SingleStatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
}

View File

@ -35,23 +35,23 @@ SingleStatisticsDescription & SingleStatisticsDescription::operator=(SingleStati
if (this == &other)
return *this;
type = std::exchange(other.type, StatisticsType{});
type = std::exchange(other.type, SingleStatisticsType{});
ast = other.ast ? other.ast->clone() : nullptr;
other.ast.reset();
return *this;
}
static StatisticsType stringToStatisticsType(String type)
static SingleStatisticsType stringToStatisticsType(String type)
{
if (type == "tdigest")
return StatisticsType::TDigest;
return SingleStatisticsType::TDigest;
if (type == "uniq")
return StatisticsType::Uniq;
return SingleStatisticsType::Uniq;
if (type == "count_min")
return StatisticsType::CountMinSketch;
return SingleStatisticsType::CountMinSketch;
if (type == "minmax")
return StatisticsType::MinMax;
return SingleStatisticsType::MinMax;
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type);
}
@ -59,20 +59,20 @@ String SingleStatisticsDescription::getTypeName() const
{
switch (type)
{
case StatisticsType::TDigest:
case SingleStatisticsType::TDigest:
return "TDigest";
case StatisticsType::Uniq:
case SingleStatisticsType::Uniq:
return "Uniq";
case StatisticsType::CountMinSketch:
case SingleStatisticsType::CountMinSketch:
return "count_min";
case StatisticsType::MinMax:
case SingleStatisticsType::MinMax:
return "minmax";
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type);
}
}
SingleStatisticsDescription::SingleStatisticsDescription(StatisticsType type_, ASTPtr ast_)
SingleStatisticsDescription::SingleStatisticsDescription(SingleStatisticsType type_, ASTPtr ast_)
: type(type_), ast(ast_)
{}

View File

@ -9,7 +9,7 @@
namespace DB
{
enum class StatisticsType : UInt8
enum class SingleStatisticsType : UInt8
{
TDigest = 0,
Uniq = 1,
@ -21,14 +21,14 @@ enum class StatisticsType : UInt8
struct SingleStatisticsDescription
{
StatisticsType type;
SingleStatisticsType type;
ASTPtr ast;
String getTypeName() const;
SingleStatisticsDescription() = delete;
SingleStatisticsDescription(StatisticsType type_, ASTPtr ast_);
SingleStatisticsDescription(SingleStatisticsType type_, ASTPtr ast_);
SingleStatisticsDescription(const SingleStatisticsDescription & other) { *this = other; }
SingleStatisticsDescription & operator=(const SingleStatisticsDescription & other);
@ -60,7 +60,7 @@ struct ColumnStatisticsDescription
static std::vector<std::pair<String, ColumnStatisticsDescription>> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type);
using StatisticsTypeDescMap = std::map<StatisticsType, SingleStatisticsDescription>;
using StatisticsTypeDescMap = std::map<SingleStatisticsType, SingleStatisticsDescription>;
StatisticsTypeDescMap types_to_desc;
DataTypePtr data_type;
};