mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Some fixups after merging
This commit is contained in:
parent
0c5821e5b8
commit
9036ce9725
@ -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}
|
||||
|
||||
|
@ -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 (...)
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -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());
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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%';
|
||||
|
Loading…
Reference in New Issue
Block a user