This commit is contained in:
JackyWoo 2024-09-19 07:41:35 +08:00 committed by GitHub
commit 208b04ce41
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 221 additions and 186 deletions

View File

@ -677,13 +677,13 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const NamesAnd
return *minimum_size_column;
}
ColumnsStatistics IMergeTreeDataPart::loadStatistics() const
Statistics IMergeTreeDataPart::loadStatistics() const
{
const auto & metadata_snaphost = storage.getInMemoryMetadata();
const auto & metadata_snapshot = storage.getInMemoryMetadata();
auto total_statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snaphost.getColumns());
auto total_statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot.getColumns());
ColumnsStatistics result;
Statistics result(rows_count);
for (auto & stat : total_statistics)
{
String file_name = stat->getFileName() + STATS_FILE_SUFFIX;
@ -691,13 +691,13 @@ ColumnsStatistics IMergeTreeDataPart::loadStatistics() const
if (!metadata_manager->exists(file_name))
{
LOG_INFO(storage.log, "Cannot find stats file {}", file_path);
LOG_TRACE(storage.log, "Cannot find statistics file {}.", file_path);
continue;
}
auto stat_file = metadata_manager->read(file_name);
CompressedReadBuffer compressed_buffer(*stat_file);
stat->deserialize(compressed_buffer);
result.push_back(stat);
result.addColumnStat(stat);
}
return result;
}

View File

@ -171,7 +171,7 @@ public:
void remove();
ColumnsStatistics loadStatistics() const;
Statistics loadStatistics() const;
/// Initialize columns (from columns.txt if exists, or create from column files if not).
/// Load various metadata into memory: checksums from checksums.txt, index if required, etc.

View File

@ -486,10 +486,8 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP
return {};
}
ASTPtr expression_ast;
ConditionSelectivityEstimator result;
PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context);
Statistics statistics;
if (partition_pruner.isUseless())
{
@ -497,11 +495,8 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP
for (const auto & part : parts)
try
{
auto stats = part->loadStatistics();
/// TODO: We only have one stats file for every part.
result.addRows(part->rows_count);
for (const auto & stat : stats)
result.merge(part->info.getPartNameV1(), stat);
auto part_statistics = part->loadStatistics();
statistics.merge(part_statistics);
}
catch (...)
{
@ -515,10 +510,8 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP
{
if (!partition_pruner.canBePruned(*part))
{
auto stats = part->loadStatistics();
result.addRows(part->rows_count);
for (const auto & stat : stats)
result.merge(part->info.getPartNameV1(), stat);
auto part_statistics = part->loadStatistics();
statistics.merge(part_statistics);
}
}
catch (...)
@ -527,7 +520,7 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP
}
}
return result;
return ConditionSelectivityEstimator(statistics);
}
bool MergeTreeData::supportsFinal() const

View File

@ -4,53 +4,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String part_name, ColumnStatisticsPtr stats)
{
if (part_statistics.contains(part_name))
throw Exception(ErrorCodes::LOGICAL_ERROR, "part {} has been added in column {}", part_name, stats->columnName());
part_statistics[part_name] = stats;
}
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(const Field & val, Float64 rows) const
{
if (part_statistics.empty())
return default_cond_range_factor * rows;
Float64 result = 0;
Float64 part_rows = 0;
for (const auto & [key, estimator] : part_statistics)
{
result += estimator->estimateLess(val);
part_rows += estimator->rowCount();
}
return result * rows / part_rows;
}
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreater(const Field & val, Float64 rows) const
{
return rows - estimateLess(val, rows);
}
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(const Field & val, Float64 rows) const
{
if (part_statistics.empty())
{
return default_cond_equal_factor * rows;
}
Float64 result = 0;
Float64 partial_cnt = 0;
for (const auto & [key, estimator] : part_statistics)
{
result += estimator->estimateEqual(val);
partial_cnt += estimator->rowCount();
}
return result * rows / partial_cnt;
}
/// second return value represents how many columns in the node.
static std::pair<String, Int32> tryToExtractSingleColumn(const RPNBuilderTreeNode & node)
{
@ -84,7 +37,7 @@ static std::pair<String, Int32> tryToExtractSingleColumn(const RPNBuilderTreeNod
return result;
}
std::pair<String, Field> ConditionSelectivityEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const
static std::pair<String, Field> extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name)
{
if (!node.isFunction())
return {};
@ -126,24 +79,16 @@ std::pair<String, Field> ConditionSelectivityEstimator::extractBinaryOp(const RP
Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode & node) const
{
auto result = tryToExtractSingleColumn(node);
auto total_rows = stats.getTotalRowCount();
if (result.second != 1)
return default_unknown_cond_factor * total_rows;
String col = result.first;
auto it = column_estimators.find(col);
/// If there the estimator of the column is not found or there are no data at all,
/// we use dummy estimation.
bool dummy = false;
ColumnSelectivityEstimator estimator;
if (it != column_estimators.end())
estimator = it->second;
else
dummy = true;
const String & col = result.first;
auto [op, val] = extractBinaryOp(node, col);
if (dummy)
/// No statistics for column col
if (const auto column_stat = stats.getColumnStat(col); column_stat == nullptr)
{
if (op == "equals")
return default_cond_equal_factor * total_rows;
@ -152,21 +97,18 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode
else
return default_unknown_cond_factor * total_rows;
}
if (op == "equals")
return estimator.estimateEqual(val, total_rows);
else if (op == "less" || op == "lessOrEquals")
return estimator.estimateLess(val, total_rows);
else if (op == "greater" || op == "greaterOrEquals")
return estimator.estimateGreater(val, total_rows);
else
return default_unknown_cond_factor * total_rows;
}
{
if (op == "equals")
return column_stat->estimateEqual(val);
else if (op == "less" || op == "lessOrEquals")
return column_stat->estimateLess(val);
else if (op == "greater" || op == "greaterOrEquals")
return column_stat->estimateGreater(val);
else
return default_unknown_cond_factor * total_rows;
}
void ConditionSelectivityEstimator::merge(String part_name, ColumnStatisticsPtr column_stat)
{
if (column_stat != nullptr)
column_estimators[column_stat->columnName()].merge(part_name, column_stat);
}
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <Storages/Statistics/Statistics.h>
#include <Core/Field.h>
namespace DB
{
@ -12,39 +11,22 @@ class RPNBuilderTreeNode;
class ConditionSelectivityEstimator
{
public:
ConditionSelectivityEstimator() : stats(Statistics()) {}
explicit ConditionSelectivityEstimator(const Statistics & stats_) : stats(stats_) {}
/// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ...
/// Right now we only support simple condition like col = val / col < val
Float64 estimateRowCount(const RPNBuilderTreeNode & node) const;
void merge(String part_name, ColumnStatisticsPtr column_stat);
void addRows(UInt64 part_rows) { total_rows += part_rows; }
private:
friend class ColumnStatistics;
struct ColumnSelectivityEstimator
{
/// We store the part_name and part_statistics.
/// then simply get selectivity for every part_statistics and combine them.
std::map<String, ColumnStatisticsPtr> part_statistics;
void merge(String part_name, ColumnStatisticsPtr stats);
Float64 estimateLess(const Field & val, Float64 rows) const;
Float64 estimateGreater(const Field & val, Float64 rows) const;
Float64 estimateEqual(const Field & val, Float64 rows) const;
};
std::pair<String, Field> extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const;
/// Used to estimate the selectivity of a condition when there is no statistics.
static constexpr auto default_cond_range_factor = 0.5;
static constexpr auto default_cond_equal_factor = 0.01;
static constexpr auto default_unknown_cond_factor = 1;
UInt64 total_rows = 0;
std::map<String, ColumnSelectivityEstimator> column_estimators;
Statistics stats;
};
}

View File

@ -54,11 +54,16 @@ std::optional<Float64> StatisticsUtils::tryConvertToFloat64(const Field & value,
return {};
}
IStatistics::IStatistics(const SingleStatisticsDescription & stat_)
ISingleStatistics::ISingleStatistics(const SingleStatisticsDescription & stat_)
: stat(stat_)
{
}
String ISingleStatistics::getTypeName() const
{
return stat.getTypeName();
}
ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_)
: stats_desc(stats_desc_), column_name(column_name_)
{
@ -71,17 +76,29 @@ void ColumnStatistics::update(const ColumnPtr & column)
stat.second->update(column);
}
UInt64 IStatistics::estimateCardinality() const
void ColumnStatistics::merge(const ColumnStatistics & other)
{
rows += other.rows;
for (const auto & [type, stat] : other.stats)
{
if (!stats.contains(type))
stats[type] = stat;
else
stats[type]->merge(stat);
}
}
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 +117,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 +131,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 +212,37 @@ UInt64 ColumnStatistics::rowCount() const
return rows;
}
void MergeTreeStatisticsFactory::registerCreator(StatisticsType stats_type, Creator creator)
void Statistics::merge(const Statistics & other)
{
total_row_count += other.total_row_count;
for (const auto & [column_name, column_stat] : other.column_stats)
{
if (!column_stats.contains(column_name))
column_stats[column_name]= column_stat;
else
column_stats[column_name]->merge(*column_stat);
}
}
void Statistics::addColumnStat(const ColumnStatisticsPtr & column_stat_)
{
column_stats[column_stat_->columnName()] = column_stat_;
}
ColumnStatisticsPtr Statistics::getColumnStat(const String & column_name) const
{
if (column_stats.contains(column_name))
return column_stats.at(column_name);
return nullptr;
}
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 +250,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,17 +19,21 @@ struct StatisticsUtils
static std::optional<Float64> tryConvertToFloat64(const Field & value, const DataTypePtr & data_type);
};
/// Statistics describe properties of the values in the column,
class ISingleStatistics;
using SingleStatisticsPtr = std::shared_ptr<ISingleStatistics>;
/// 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;
virtual void merge(const SingleStatisticsPtr & other) = 0;
virtual void serialize(WriteBuffer & buf) = 0;
virtual void deserialize(ReadBuffer & buf) = 0;
@ -45,11 +47,12 @@ public:
virtual Float64 estimateEqual(const Field & val) const; /// cardinality of val in the column
virtual Float64 estimateLess(const Field & val) const; /// summarized cardinality of values < val in the column
String getTypeName() const;
protected:
SingleStatisticsDescription stat;
};
using StatisticsPtr = std::shared_ptr<IStatistics>;
class ColumnStatistics
{
@ -65,6 +68,7 @@ public:
UInt64 rowCount() const;
void update(const ColumnPtr & column);
void merge(const ColumnStatistics & other);
Float64 estimateLess(const Field & val) const;
Float64 estimateGreater(const Field & val) const;
@ -74,7 +78,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
};
@ -83,6 +87,36 @@ class ColumnsDescription;
using ColumnStatisticsPtr = std::shared_ptr<ColumnStatistics>;
using ColumnsStatistics = std::vector<ColumnStatisticsPtr>;
/// Statistics represent the statistics for a part, table or node in query plan.
/// It is a computable data structure which means:
/// 1. it can accumulate statistics for a dedicated column of multiple parts.
/// 2. it can support more calculations which is needed in estimating complex predicates like `a > 1 and a < 100 or a in (1, 3, 500).
/// For example column 'a' has a minmax statistics with range [1, 100], after estimate 'a>50' the range is [51, 100].
class Statistics
{
public:
explicit Statistics() : total_row_count(0) { }
explicit Statistics(UInt64 total_row_count_) : total_row_count(total_row_count_) { }
explicit Statistics(UInt64 total_row_count_, const std::unordered_map<String, ColumnStatisticsPtr> & column_stats_)
: total_row_count(total_row_count_)
, column_stats(column_stats_)
{
}
void merge(const Statistics & other);
void addColumnStat(const ColumnStatisticsPtr & column_stat_);
UInt64 getTotalRowCount() const { return total_row_count; }
ColumnStatisticsPtr getColumnStat(const String & column_name) const;
private:
UInt64 total_row_count;
/// column_name -> column_statistics
std::unordered_map<String, ColumnStatisticsPtr> column_stats;
};
using StatisticsPtr = std::shared_ptr<Statistics>;
class MergeTreeStatisticsFactory : private boost::noncopyable
{
public:
@ -91,20 +125,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

@ -17,7 +17,7 @@ extern const int ILLEGAL_STATISTICS;
}
/// Constants chosen based on rolling dices.
/// The values provides:
/// The values provide:
/// 1. an error tolerance of 0.1% (ε = 0.001)
/// 2. a confidence level of 99.9% (δ = 0.001).
/// And sketch the size is 152kb.
@ -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_)
{
@ -63,6 +63,14 @@ void StatisticsCountMinSketch::update(const ColumnPtr & column)
}
}
void StatisticsCountMinSketch::merge(const SingleStatisticsPtr & other)
{
if (const auto * other_stat = dynamic_cast<const StatisticsCountMinSketch *>(other.get()))
sketch.merge(other_stat->sketch);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to merge statistics of type {} to CountMin statistics", toString(other->getTypeName()));
}
void StatisticsCountMinSketch::serialize(WriteBuffer & buf)
{
Sketch::vector_bytes bytes = sketch.serialize();
@ -91,7 +99,7 @@ void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*des
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'countmin' 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_);
@ -19,6 +19,7 @@ public:
Float64 estimateEqual(const Field & val) const override;
void update(const ColumnPtr & column) override;
void merge(const SingleStatisticsPtr & other) override;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
@ -32,7 +33,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

@ -13,10 +13,11 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_STATISTICS;
extern const int LOGICAL_ERROR;
}
StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & description, const DataTypePtr & data_type_)
: IStatistics(description)
: ISingleStatistics(description)
, data_type(data_type_)
{
}
@ -35,6 +36,18 @@ void StatisticsMinMax::update(const ColumnPtr & column)
row_count += column->size();
}
void StatisticsMinMax::merge(const SingleStatisticsPtr & other)
{
if (const auto * other_stat = dynamic_cast<const StatisticsMinMax *>(other.get()))
{
min = std::min(other_stat->min, min);
max = std::min(other_stat->max, max);
row_count += other_stat->row_count;
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to merge statistics of type {} to MinMax statistics", toString(other->getTypeName()));
}
void StatisticsMinMax::serialize(WriteBuffer & buf)
{
writeIntBinary(row_count, buf);
@ -78,7 +91,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,12 +7,13 @@
namespace DB
{
class StatisticsMinMax : public IStatistics
class StatisticsMinMax : public ISingleStatistics
{
public:
StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_);
void update(const ColumnPtr & column) override;
void merge(const SingleStatisticsPtr & other) override;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
@ -28,6 +29,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

@ -7,10 +7,11 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_STATISTICS;
extern const int LOGICAL_ERROR;
}
StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_)
: IStatistics(description)
: ISingleStatistics(description)
, data_type(data_type_)
{
}
@ -27,6 +28,14 @@ void StatisticsTDigest::update(const ColumnPtr & column)
}
}
void StatisticsTDigest::merge(const SingleStatisticsPtr & other)
{
if (const auto * other_stat = dynamic_cast<const StatisticsTDigest *>(other.get()))
t_digest.merge(other_stat->t_digest);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to merge statistics of type {} to TDigest statistics", toString(other->getTypeName()));
}
void StatisticsTDigest::serialize(WriteBuffer & buf)
{
t_digest.serialize(buf);
@ -61,7 +70,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,12 +6,13 @@
namespace DB
{
class StatisticsTDigest : public IStatistics
class StatisticsTDigest : public ISingleStatistics
{
public:
explicit StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_);
void update(const ColumnPtr & column) override;
void merge(const SingleStatisticsPtr & other) override;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
@ -25,6 +26,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

@ -9,10 +9,11 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_STATISTICS;
extern const int LOGICAL_ERROR;
}
StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type)
: IStatistics(description)
: ISingleStatistics(description)
{
arena = std::make_unique<Arena>();
AggregateFunctionProperties properties;
@ -35,6 +36,14 @@ void StatisticsUniq::update(const ColumnPtr & column)
collector->addBatchSinglePlace(0, column->size(), data, &(raw_ptr), nullptr);
}
void StatisticsUniq::merge(const SingleStatisticsPtr & other)
{
if (const auto * other_stat = dynamic_cast<const StatisticsUniq *>(other.get()))
collector->merge(other_stat->data, data, arena.get());
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to merge statistics of type {} to Uniq statistics", toString(other->getTypeName()));
}
void StatisticsUniq::serialize(WriteBuffer & buf)
{
collector->serialize(data, buf);
@ -60,7 +69,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,13 +7,14 @@
namespace DB
{
class StatisticsUniq : public IStatistics
class StatisticsUniq : public ISingleStatistics
{
public:
StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type);
~StatisticsUniq() override;
void update(const ColumnPtr & column) override;
void merge(const SingleStatisticsPtr & other) override;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
@ -28,6 +29,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 == "countmin")
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 'countmin', '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 "countmin";
case StatisticsType::MinMax:
case SingleStatisticsType::MinMax:
return "minmax";
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'countmin', '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;
};