Some fixups after merging

This commit is contained in:
JackyWoo 2024-07-04 11:09:47 +08:00
parent 0c5821e5b8
commit 9036ce9725
13 changed files with 65 additions and 76 deletions

View File

@ -1001,7 +1001,7 @@ They can be used for prewhere optimization only if we enable `set allow_statisti
- `count_min`
[count min sketch](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.
[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.
## Column-level Settings {#column-level-settings}

View File

@ -496,11 +496,9 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP
{
auto stats = part->loadStatistics();
/// TODO: We only have one stats file for every part.
if (stats.empty()) /// No statistics still need add rows count.
result.addRows(part->rows_count);
else
for (const auto & stat : stats)
result.merge(part->info.getPartNameV1(), part->rows_count, stat);
result.addRows(part->rows_count);
for (const auto & stat : stats)
result.merge(part->info.getPartNameV1(), stat);
}
catch (...)
{
@ -515,11 +513,9 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP
if (!partition_pruner.canBePruned(*part))
{
auto stats = part->loadStatistics();
if (stats.empty())
result.addRows(part->rows_count);
else
for (const auto & stat : stats)
result.merge(part->info.getPartNameV1(), part->rows_count, stat);
result.addRows(part->rows_count);
for (const auto & stat : stats)
result.merge(part->info.getPartNameV1(), stat);
}
}
catch (...)

View File

@ -35,7 +35,7 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreat
return rows - estimateLess(val, rows);
}
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(Field val, Float64 rows) const
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(const Field & val, Float64 rows) const
{
auto float_val = IStatistics::getFloat64(val);
if (part_statistics.empty())
@ -141,7 +141,7 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode
/// If there the estimator of the column is not found or there are no data at all,
/// we use dummy estimation.
bool dummy = total_rows == 0;
bool dummy = false;
ColumnSelectivityEstimator estimator;
if (it != column_estimators.end())
estimator = it->second;
@ -176,13 +176,8 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode
return default_unknown_cond_factor * total_rows;
}
void ConditionSelectivityEstimator::merge(String part_name, UInt64 part_rows, ColumnStatisticsPtr column_stat)
void ConditionSelectivityEstimator::merge(String part_name, ColumnStatisticsPtr column_stat)
{
if (!part_names.contains(part_name))
{
total_rows += part_rows;
part_names.insert(part_name);
}
if (column_stat != nullptr)
column_estimators[column_stat->columnName()].merge(part_name, column_stat);
}

View File

@ -11,6 +11,14 @@ class RPNBuilderTreeNode;
/// It estimates the selectivity of a condition.
class ConditionSelectivityEstimator
{
public:
/// 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
@ -25,9 +33,11 @@ private:
Float64 estimateGreater(Float64 val, Float64 rows) const;
Float64 estimateEqual(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;
static constexpr auto default_good_cond_factor = 0.1;
static constexpr auto default_normal_cond_factor = 0.5;
static constexpr auto default_unknown_cond_factor = 1.0;
@ -36,17 +46,7 @@ private:
static constexpr auto threshold = 2;
UInt64 total_rows = 0;
std::set<String> part_names;
std::map<String, ColumnSelectivityEstimator> column_estimators;
std::pair<String, Field> extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const;
public:
/// 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, UInt64 part_rows, ColumnStatisticsPtr column_stat);
void addRows(UInt64 part_rows) { total_rows += part_rows; }
};
}

View File

@ -1,12 +1,12 @@
#include <Storages/Statistics/Statistics.h>
#include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <Storages/Statistics/StatisticsTDigest.h>
#include <Storages/Statistics/StatisticsUniq.h>
#include <Storages/Statistics/CountMinSketchStatistics.h>
#include <Storages/StatisticsDescription.h>
#include <Storages/ColumnsDescription.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <Storages/Statistics/Statistics.h>
#include <Storages/Statistics/StatisticsCountMinSketch.h>
#include <Storages/Statistics/StatisticsTDigest.h>
#include <Storages/Statistics/StatisticsUniq.h>
#include <Storages/StatisticsDescription.h>
#include <Common/Exception.h>
#include <Common/logger_useful.h>
@ -93,7 +93,7 @@ UInt64 IStatistics::estimateCardinality() const
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cardinality estimation is not implemented for this type of statistics");
}
Float64 IStatistics::estimateEqual(Float64 /*val*/) const
Float64 IStatistics::estimateEqual(const Field & /*val*/) const
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Equality estimation is not implemented for this type of statistics");
}
@ -125,23 +125,20 @@ Float64 ColumnStatistics::estimateGreater(Float64 val) const
return rows - estimateLess(val);
}
Float64 ColumnStatistics::estimateEqual(Field val) const
Float64 ColumnStatistics::estimateEqual(const Field & val) const
{
auto float_val = IStatistics::getFloat64(val);
if (stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest))
if (float_val.has_value() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::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(float_val);
return stats.at(StatisticsType::TDigest)->estimateEqual(val);
}
#if USE_DATASKETCHES
if (stats.contains(StatisticsType::CountMinSketch))
{
auto count_min_sketch_static = std::static_pointer_cast<CountMinSketchStatistics>(stats.at(StatisticsType::CountMinSketch));
return count_min_sketch_static->estimateEqual(val);
}
return stats.at(StatisticsType::CountMinSketch)->estimateEqual(val);
#endif
if (float_val < - ConditionSelectivityEstimator::threshold || float_val > ConditionSelectivityEstimator::threshold)
if (!float_val.has_value() && (float_val < - ConditionSelectivityEstimator::threshold || float_val > ConditionSelectivityEstimator::threshold))
return rows * ConditionSelectivityEstimator::default_normal_cond_factor;
else
return rows * ConditionSelectivityEstimator::default_good_cond_factor;

View File

@ -35,7 +35,7 @@ public:
/// Per-value estimations.
/// Throws if the statistics object is not able to do a meaningful estimation.
virtual Float64 estimateEqual(Float64 val) const; /// cardinality of val in the column
virtual Float64 estimateEqual(const Field & val) const; /// cardinality of val in the column
virtual Float64 estimateLess(Float64 val) const; /// summarized cardinality of values < val in the column
/// Convert filed to Float64, used when estimating the number of rows.
@ -67,8 +67,7 @@ public:
Float64 estimateLess(Float64 val) const;
Float64 estimateGreater(Float64 val) const;
Float64 estimateEqual(Float64 val) const;
Float64 estimateEqual(Field val) const;
Float64 estimateEqual(const Field & val) const;
private:
friend class MergeTreeStatisticsFactory;

View File

@ -1,8 +1,8 @@
#include <Storages/Statistics/CountMinSketchStatistics.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Storages/Statistics/StatisticsCountMinSketch.h>
#if USE_DATASKETCHES
@ -14,15 +14,17 @@ namespace ErrorCodes
extern const int ILLEGAL_STATISTICS;
}
static constexpr auto num_hashes = 8uz;
static constexpr auto num_buckets = 2048uz;
CountMinSketchStatistics::CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_)
StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_)
: IStatistics(stat_)
, sketch(num_hashes, num_buckets)
, data_type(data_type_)
{
}
Float64 CountMinSketchStatistics::estimateEqual(const Field & value) const
Float64 StatisticsCountMinSketch::estimateEqual(const Field & value) const
{
if (auto float_val = IStatistics::getFloat64(value))
return sketch.get_estimate(&float_val.value(), 8);
@ -31,14 +33,14 @@ Float64 CountMinSketchStatistics::estimateEqual(const Field & value) const
UNREACHABLE();
}
void CountMinSketchStatistics::serialize(WriteBuffer & buf)
void StatisticsCountMinSketch::serialize(WriteBuffer & buf)
{
Sketch::vector_bytes bytes = sketch.serialize();
writeIntBinary(static_cast<UInt64>(bytes.size()), buf);
buf.write(reinterpret_cast<const char *>(bytes.data()), bytes.size());
}
void CountMinSketchStatistics::deserialize(ReadBuffer & buf)
void StatisticsCountMinSketch::deserialize(ReadBuffer & buf)
{
UInt64 size;
readIntBinary(size, buf);
@ -47,10 +49,10 @@ void CountMinSketchStatistics::deserialize(ReadBuffer & buf)
bytes.reserve(size);
buf.readStrict(reinterpret_cast<char *>(bytes.data()), size);
sketch = datasketches::count_min_sketch<UInt64>::deserialize(bytes.data(), size);
sketch = Sketch::deserialize(bytes.data(), size);
}
void CountMinSketchStatistics::update(const ColumnPtr & column)
void StatisticsCountMinSketch::update(const ColumnPtr & column)
{
size_t size = column->size();
for (size_t i = 0; i < size; ++i)
@ -78,7 +80,7 @@ void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da
StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type)
{
return std::make_shared<CountMinSketchStatistics>(stat, data_type);
return std::make_shared<StatisticsCountMinSketch>(stat, data_type);
}
}

View File

@ -11,12 +11,12 @@
namespace DB
{
class CountMinSketchStatistics : public IStatistics
class StatisticsCountMinSketch : public IStatistics
{
public:
CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_);
StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_);
Float64 estimateEqual(const Field & value) const;
Float64 estimateEqual(const Field & value) const override;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
@ -24,9 +24,6 @@ public:
void update(const ColumnPtr & column) override;
private:
static constexpr auto num_hashes = 8uz;
static constexpr auto num_buckets = 2048uz;
using Sketch = datasketches::count_min_sketch<UInt64>;
Sketch sketch;

View File

@ -1,5 +1,6 @@
#include <Storages/Statistics/StatisticsTDigest.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace DB
{
@ -16,12 +17,16 @@ StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_)
void StatisticsTDigest::update(const ColumnPtr & column)
{
size_t rows = column->size();
for (size_t row = 0; row < rows; ++row)
{
/// TODO: support more types.
Float64 value = column->getFloat64(row);
t_digest.add(value, 1);
Field f;
column->get(row, f);
if (f.isNull())
continue;
if (auto float_val = IStatistics::getFloat64(f))
t_digest.add(*float_val, 1);
}
}
@ -40,14 +45,15 @@ Float64 StatisticsTDigest::estimateLess(Float64 val) const
return t_digest.getCountLessThan(val);
}
Float64 StatisticsTDigest::estimateEqual(Float64 val) const
Float64 StatisticsTDigest::estimateEqual(const Field & val) const
{
return t_digest.getCountEqual(val);
return t_digest.getCountEqual(IStatistics::getFloat64(val).value());
}
void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
{
data_type = removeNullable(data_type);
data_type = removeLowCardinalityAndNullable(data_type);
if (!data_type->isValueRepresentedByNumber())
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName());
}

View File

@ -17,7 +17,7 @@ public:
void deserialize(ReadBuffer & buf) override;
Float64 estimateLess(Float64 val) const override;
Float64 estimateEqual(Float64 val) const override;
Float64 estimateEqual(const Field & val) const override;
private:
QuantileTDigest<Float64> t_digest;

View File

@ -1,19 +1,14 @@
#include <Storages/StatisticsDescription.h>
#include <base/defines.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Parsers/ParserCreateQuery.h>
#include <Poco/Logger.h>
#include <Storages/extractKeyExpressionList.h>
#include <Storages/ColumnsDescription.h>
#include <Common/logger_useful.h>
namespace DB
{

View File

@ -27,6 +27,7 @@ ALTER TABLE t1 MATERIALIZE STATISTICS b, c;
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '')
FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b > 0/*9990*/ and c < -98/*100*/)
WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '')
FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b = 0/*1000*/ and c < -98/*100*/)
WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
@ -69,6 +70,7 @@ ALTER TABLE t1 MATERIALIZE STATISTICS a, b, c;
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '')
FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/)
WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '')
FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/)
WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';