From 15e20f56fa4a9a497e4cc247aa9215cb840203f9 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 21 Jun 2024 16:29:25 +0800 Subject: [PATCH 01/59] Add statistics cmsketch --- contrib/datasketches-cpp-cmake/CMakeLists.txt | 1 + .../statements/alter/statistics.md | 2 +- src/CMakeLists.txt | 1 + .../Statistics/CMSketchStatistics.cpp | 82 +++++++++++++++++++ src/Storages/Statistics/CMSketchStatistics.h | 39 +++++++++ .../ConditionSelectivityEstimator.cpp | 35 +++----- .../ConditionSelectivityEstimator.h | 5 +- src/Storages/Statistics/Statistics.cpp | 41 +++++++++- src/Storages/Statistics/Statistics.h | 5 +- src/Storages/StatisticsDescription.cpp | 4 + src/Storages/StatisticsDescription.h | 1 + .../03174_statistics_cmsketch.reference | 26 ++++++ .../0_stateless/03174_statistics_cmsketch.sql | 78 ++++++++++++++++++ 13 files changed, 290 insertions(+), 30 deletions(-) create mode 100644 src/Storages/Statistics/CMSketchStatistics.cpp create mode 100644 src/Storages/Statistics/CMSketchStatistics.h create mode 100644 tests/queries/0_stateless/03174_statistics_cmsketch.reference create mode 100644 tests/queries/0_stateless/03174_statistics_cmsketch.sql diff --git a/contrib/datasketches-cpp-cmake/CMakeLists.txt b/contrib/datasketches-cpp-cmake/CMakeLists.txt index b12a88ad57b..497d6956d0e 100644 --- a/contrib/datasketches-cpp-cmake/CMakeLists.txt +++ b/contrib/datasketches-cpp-cmake/CMakeLists.txt @@ -9,6 +9,7 @@ set(DATASKETCHES_LIBRARY theta) add_library(_datasketches INTERFACE) target_include_directories(_datasketches SYSTEM BEFORE INTERFACE "${ClickHouse_SOURCE_DIR}/contrib/datasketches-cpp/common/include" + "${ClickHouse_SOURCE_DIR}/contrib/datasketches-cpp/count/include" "${ClickHouse_SOURCE_DIR}/contrib/datasketches-cpp/theta/include") add_library(ch_contrib::datasketches ALIAS _datasketches) diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 80024781f88..22ff740d410 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -25,7 +25,7 @@ Also, they are replicated, syncing statistics metadata via ZooKeeper. There is an example adding two statistics types to two columns: ``` -ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; +ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq, CMSketch; ``` :::note diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 84aaec17a5b..4efb6004172 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -546,6 +546,7 @@ endif() if (TARGET ch_contrib::datasketches) target_link_libraries (clickhouse_aggregate_functions PRIVATE ch_contrib::datasketches) + dbms_target_link_libraries(PRIVATE ch_contrib::datasketches) endif () target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) diff --git a/src/Storages/Statistics/CMSketchStatistics.cpp b/src/Storages/Statistics/CMSketchStatistics.cpp new file mode 100644 index 00000000000..2c217d30278 --- /dev/null +++ b/src/Storages/Statistics/CMSketchStatistics.cpp @@ -0,0 +1,82 @@ +#include +#include +#include +#include +#include + +#if USE_DATASKETCHES + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ILLEGAL_STATISTICS; +} + + +CMSketchStatistics::CMSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) + : IStatistics(stat_), data(CMSKETCH_HASH_COUNT, CMSKETCH_BUCKET_COUNT), data_type(data_type_) +{ +} + +Float64 CMSketchStatistics::estimateEqual(const Field & value) const +{ + if (auto float_val = getFloat64(value)) + return data.get_estimate(&float_val.value(), 8); + if (auto string_val = getString(value)) + return data.get_estimate(string_val->data(), string_val->size()); + UNREACHABLE(); +} + +void CMSketchStatistics::serialize(WriteBuffer & buf) +{ + auto bytes = data.serialize(); + writeIntBinary(static_cast(bytes.size()), buf); + buf.write(reinterpret_cast(bytes.data()), bytes.size()); +} + +void CMSketchStatistics::deserialize(ReadBuffer & buf) +{ + UInt64 size; + readIntBinary(size, buf); + String s; + s.reserve(size); + buf.readStrict(s.data(), size); /// Extra copy can be avoided by implementing count_min_sketch::deserialize with ReadBuffer + auto read_sketch = datasketches::count_min_sketch::deserialize(s.data(), size, datasketches::DEFAULT_SEED); + data.merge(read_sketch); +} + +void CMSketchStatistics::update(const ColumnPtr & column) +{ + size_t size = column->size(); + + for (size_t i = 0; i < size; ++i) + { + Field f; + column->get(i, f); + if (f.isNull()) + continue; + if (auto float_val = getFloat64(f)) + data.update(&float_val.value(), 8, 1.0); + else if (auto string_val = getString(f)) + data.update(*string_val, 1.0); + } +} + +void CMSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +{ + data_type = removeNullable(data_type); + data_type = removeLowCardinalityAndNullable(data_type); + if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'cmsketch' does not support type {}", data_type->getName()); +} + +StatisticsPtr CMSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +{ + return std::make_shared(stat, data_type); +} + +} + +#endif diff --git a/src/Storages/Statistics/CMSketchStatistics.h b/src/Storages/Statistics/CMSketchStatistics.h new file mode 100644 index 00000000000..03964614d57 --- /dev/null +++ b/src/Storages/Statistics/CMSketchStatistics.h @@ -0,0 +1,39 @@ +#pragma once + +#if USE_DATASKETCHES + +#include +#include +#include + +namespace DB +{ + +/// CMSketchStatistics is used to estimate expression like col = 'value' or col in ('v1', 'v2'). +class CMSketchStatistics : public IStatistics +{ +public: + explicit CMSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); + + Float64 estimateEqual(const Field & value) const; + + void serialize(WriteBuffer & buf) override; + + void deserialize(ReadBuffer & buf) override; + + void update(const ColumnPtr & column) override; + +private: + static constexpr size_t CMSKETCH_HASH_COUNT = 8; + static constexpr size_t CMSKETCH_BUCKET_COUNT = 2048; + + datasketches::count_min_sketch data; + DataTypePtr data_type; +}; + +StatisticsPtr CMSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void CMSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); + +} + +#endif diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 757136fdf42..73c5c549a5d 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -35,11 +35,14 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreat return rows - estimateLess(val, rows); } -Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(Float64 val, Float64 rows) const +Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(Field val, Float64 rows) const { + auto float_val = getFloat64(val); if (part_statistics.empty()) { - if (val < - threshold || val > threshold) + if (!float_val) + return default_unknown_cond_factor * rows; + else if (float_val.value() < - threshold || float_val.value() > threshold) return default_normal_cond_factor * rows; else return default_good_cond_factor * rows; @@ -87,7 +90,7 @@ static std::pair tryToExtractSingleColumn(const RPNBuilderTreeNod return result; } -std::pair ConditionSelectivityEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const +std::pair ConditionSelectivityEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const { if (!node.isFunction()) return {}; @@ -123,18 +126,7 @@ std::pair ConditionSelectivityEstimator::extractBinaryOp(const DataTypePtr output_type; if (!constant_node->tryGetConstant(output_value, output_type)) return {}; - - const auto type = output_value.getType(); - Float64 value; - if (type == Field::Types::Int64) - value = output_value.get(); - else if (type == Field::Types::UInt64) - value = output_value.get(); - else if (type == Field::Types::Float64) - value = output_value.get(); - else - return {}; - return std::make_pair(function_name, value); + return std::make_pair(function_name, output_value); } Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode & node) const @@ -142,7 +134,7 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode auto result = tryToExtractSingleColumn(node); if (result.second != 1) { - return default_unknown_cond_factor; + return default_unknown_cond_factor * total_rows; } String col = result.first; auto it = column_estimators.find(col); @@ -152,19 +144,16 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode bool dummy = total_rows == 0; ColumnSelectivityEstimator estimator; if (it != column_estimators.end()) - { estimator = it->second; - } else - { dummy = true; - } auto [op, val] = extractBinaryOp(node, col); + auto float_val = getFloat64(val); if (op == "equals") { if (dummy) { - if (val < - threshold || val > threshold) + if (!float_val || (float_val < - threshold || float_val > threshold)) return default_normal_cond_factor * total_rows; else return default_good_cond_factor * total_rows; @@ -175,13 +164,13 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode { if (dummy) return default_normal_cond_factor * total_rows; - return estimator.estimateLess(val, total_rows); + return estimator.estimateLess(float_val.value(), total_rows); } else if (op == "greater" || op == "greaterOrEquals") { if (dummy) return default_normal_cond_factor * total_rows; - return estimator.estimateGreater(val, total_rows); + return estimator.estimateGreater(float_val.value(), total_rows); } else return default_unknown_cond_factor * total_rows; diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index f0599742276..9bf4940e563 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -24,7 +25,7 @@ private: Float64 estimateGreater(Float64 val, Float64 rows) const; - Float64 estimateEqual(Float64 val, Float64 rows) const; + Float64 estimateEqual(Field val, Float64 rows) const; }; static constexpr auto default_good_cond_factor = 0.1; @@ -37,7 +38,7 @@ private: UInt64 total_rows = 0; std::set part_names; std::map column_estimators; - std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const; + std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const; public: /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index fed0bd61c03..b35d653b7ec 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +27,28 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; +std::optional getFloat64(const Field & f) +{ + const auto type = f.getType(); + Float64 value; + if (type == Field::Types::Int64) + value = f.get(); + else if (type == Field::Types::UInt64) + value = f.get(); + else if (type == Field::Types::Float64) + value = f.get(); + else + return {}; + return value; +} + +std::optional getString(const Field & f) +{ + if (f.getType() == Field::Types::String) + return f.get(); + return {}; +} + IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) {} ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_) @@ -54,9 +77,10 @@ Float64 ColumnStatistics::estimateGreater(Float64 val) const return rows - estimateLess(val); } -Float64 ColumnStatistics::estimateEqual(Float64 val) const +Float64 ColumnStatistics::estimateEqual(Field val) const { - if (stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) + auto float_val = getFloat64(val); + if (float_val && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) { auto uniq_static = std::static_pointer_cast(stats.at(StatisticsType::Uniq)); /// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows) @@ -64,9 +88,16 @@ Float64 ColumnStatistics::estimateEqual(Float64 val) const if (uniq_static->getCardinality() < 2048) { auto tdigest_static = std::static_pointer_cast(stats.at(StatisticsType::TDigest)); - return tdigest_static->estimateEqual(val); + return tdigest_static->estimateEqual(float_val.value()); } } +#if USE_DATASKETCHES + if (stats.contains(StatisticsType::CMSketch)) + { + auto cmsketch_static = std::static_pointer_cast(stats.at(StatisticsType::CMSketch)); + return cmsketch_static->estimateEqual(val); + } +#endif if (val < - ConditionSelectivityEstimator::threshold || val > ConditionSelectivityEstimator::threshold) return rows * ConditionSelectivityEstimator::default_normal_cond_factor; else @@ -145,6 +176,10 @@ MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() registerCreator(StatisticsType::Uniq, UniqCreator); registerValidator(StatisticsType::TDigest, TDigestValidator); registerValidator(StatisticsType::Uniq, UniqValidator); +#if USE_DATASKETCHES + registerCreator(StatisticsType::CMSketch, CMSketchCreator); + registerValidator(StatisticsType::CMSketch, CMSketchValidator); +#endif } MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance() diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 2ab1337af02..f6121d72256 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -58,7 +59,7 @@ public: Float64 estimateGreater(Float64 val) const; - Float64 estimateEqual(Float64 val) const; + Float64 estimateEqual(Field val) const; private: @@ -100,4 +101,6 @@ private: Validators validators; }; +std::optional getFloat64(const Field & f); +std::optional getString(const Field & f); } diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index f10fb78f933..08c79043ac4 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -54,6 +54,8 @@ static StatisticsType stringToStatisticsType(String type) return StatisticsType::TDigest; if (type == "uniq") return StatisticsType::Uniq; + if (type == "cmsketch") + return StatisticsType::CMSketch; throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); } @@ -65,6 +67,8 @@ String SingleStatisticsDescription::getTypeName() const return "TDigest"; case StatisticsType::Uniq: return "Uniq"; + case StatisticsType::CMSketch: + return "CMSketch"; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 4862fb79d45..a2005f59de1 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -13,6 +13,7 @@ enum class StatisticsType : UInt8 { TDigest = 0, Uniq = 1, + CMSketch = 2, Max = 63, }; diff --git a/tests/queries/0_stateless/03174_statistics_cmsketch.reference b/tests/queries/0_stateless/03174_statistics_cmsketch.reference new file mode 100644 index 00000000000..ea77f317a31 --- /dev/null +++ b/tests/queries/0_stateless/03174_statistics_cmsketch.reference @@ -0,0 +1,26 @@ +CREATE TABLE default.t1\n(\n `a` String STATISTICS(cmsketch),\n `b` Int64 STATISTICS(cmsketch),\n `c` UInt64 STATISTICS(cmsketch),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'_String), equals(c, 0), greater(b, 0)) (removed) +After drop statistics for a + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(b, 0), equals(c, 0), equals(a, \'0\'_String)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 0), equals(a, \'0\'_String), greater(b, 0)) (removed) +LowCardinality + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) +Nullable + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) +LowCardinality(Nullable) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) diff --git a/tests/queries/0_stateless/03174_statistics_cmsketch.sql b/tests/queries/0_stateless/03174_statistics_cmsketch.sql new file mode 100644 index 00000000000..c45d6186fdf --- /dev/null +++ b/tests/queries/0_stateless/03174_statistics_cmsketch.sql @@ -0,0 +1,78 @@ +-- Tags: no-fasttest +DROP TABLE IF EXISTS t1; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +CREATE TABLE t1 +( + a String STATISTICS(cmsketch), + b Int64 STATISTICS(cmsketch), + c UInt64 STATISTICS(cmsketch), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b > 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE t1 DROP STATISTICS a; + +SELECT 'After drop statistics for a'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b > 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE t2 +( + a LowCardinality(String) STATISTICS(cmsketch), + b Int64 STATISTICS(cmsketch), + c UInt64 STATISTICS(cmsketch), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t2 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'LowCardinality'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t2 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + + +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t3 +( + a Nullable(String) STATISTICS(cmsketch), + b Int64 STATISTICS(cmsketch), + c UInt64 STATISTICS(cmsketch), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t3 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'Nullable'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t3 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t3; +DROP TABLE IF EXISTS t4; + +CREATE TABLE t4 +( + a LowCardinality(Nullable(String)) STATISTICS(cmsketch), + b Int64 STATISTICS(cmsketch), + c UInt64 STATISTICS(cmsketch), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t4 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'LowCardinality(Nullable)'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t4 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t4; From a9331c3bcdb6911497fca06c0db5ebb1afd7b2a7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 24 Jun 2024 18:26:38 +0800 Subject: [PATCH 02/59] Rname cmsketch to countmixsketch --- .../statements/alter/statistics.md | 2 +- src/Storages/Statistics/CMSketchStatistics.h | 39 ------------------- ...stics.cpp => CountMinSketchStatistics.cpp} | 24 ++++++------ .../Statistics/CountMinSketchStatistics.h | 39 +++++++++++++++++++ src/Storages/Statistics/Statistics.cpp | 24 ++++++------ src/Storages/StatisticsDescription.cpp | 8 ++-- src/Storages/StatisticsDescription.h | 2 +- ...03174_statistics_countminsketch.reference} | 2 +- ...ql => 03174_statistics_countminsketch.sql} | 24 ++++++------ 9 files changed, 82 insertions(+), 82 deletions(-) delete mode 100644 src/Storages/Statistics/CMSketchStatistics.h rename src/Storages/Statistics/{CMSketchStatistics.cpp => CountMinSketchStatistics.cpp} (66%) create mode 100644 src/Storages/Statistics/CountMinSketchStatistics.h rename tests/queries/0_stateless/{03174_statistics_cmsketch.reference => 03174_statistics_countminsketch.reference} (80%) rename tests/queries/0_stateless/{03174_statistics_cmsketch.sql => 03174_statistics_countminsketch.sql} (84%) diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 22ff740d410..0d1fa59cf86 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -25,7 +25,7 @@ Also, they are replicated, syncing statistics metadata via ZooKeeper. There is an example adding two statistics types to two columns: ``` -ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq, CMSketch; +ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq, CountMinSketch; ``` :::note diff --git a/src/Storages/Statistics/CMSketchStatistics.h b/src/Storages/Statistics/CMSketchStatistics.h deleted file mode 100644 index 03964614d57..00000000000 --- a/src/Storages/Statistics/CMSketchStatistics.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#if USE_DATASKETCHES - -#include -#include -#include - -namespace DB -{ - -/// CMSketchStatistics is used to estimate expression like col = 'value' or col in ('v1', 'v2'). -class CMSketchStatistics : public IStatistics -{ -public: - explicit CMSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); - - Float64 estimateEqual(const Field & value) const; - - void serialize(WriteBuffer & buf) override; - - void deserialize(ReadBuffer & buf) override; - - void update(const ColumnPtr & column) override; - -private: - static constexpr size_t CMSKETCH_HASH_COUNT = 8; - static constexpr size_t CMSKETCH_BUCKET_COUNT = 2048; - - datasketches::count_min_sketch data; - DataTypePtr data_type; -}; - -StatisticsPtr CMSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); -void CMSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); - -} - -#endif diff --git a/src/Storages/Statistics/CMSketchStatistics.cpp b/src/Storages/Statistics/CountMinSketchStatistics.cpp similarity index 66% rename from src/Storages/Statistics/CMSketchStatistics.cpp rename to src/Storages/Statistics/CountMinSketchStatistics.cpp index 2c217d30278..d0f52e3f6df 100644 --- a/src/Storages/Statistics/CMSketchStatistics.cpp +++ b/src/Storages/Statistics/CountMinSketchStatistics.cpp @@ -1,8 +1,8 @@ -#include #include +#include #include #include -#include +#include #if USE_DATASKETCHES @@ -15,12 +15,12 @@ extern const int ILLEGAL_STATISTICS; } -CMSketchStatistics::CMSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_), data(CMSKETCH_HASH_COUNT, CMSKETCH_BUCKET_COUNT), data_type(data_type_) +CountMinSketchStatistics::CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) + : IStatistics(stat_), data(HASH_COUNT, BUCKET_COUNT), data_type(data_type_) { } -Float64 CMSketchStatistics::estimateEqual(const Field & value) const +Float64 CountMinSketchStatistics::estimateEqual(const Field & value) const { if (auto float_val = getFloat64(value)) return data.get_estimate(&float_val.value(), 8); @@ -29,14 +29,14 @@ Float64 CMSketchStatistics::estimateEqual(const Field & value) const UNREACHABLE(); } -void CMSketchStatistics::serialize(WriteBuffer & buf) +void CountMinSketchStatistics::serialize(WriteBuffer & buf) { auto bytes = data.serialize(); writeIntBinary(static_cast(bytes.size()), buf); buf.write(reinterpret_cast(bytes.data()), bytes.size()); } -void CMSketchStatistics::deserialize(ReadBuffer & buf) +void CountMinSketchStatistics::deserialize(ReadBuffer & buf) { UInt64 size; readIntBinary(size, buf); @@ -47,7 +47,7 @@ void CMSketchStatistics::deserialize(ReadBuffer & buf) data.merge(read_sketch); } -void CMSketchStatistics::update(const ColumnPtr & column) +void CountMinSketchStatistics::update(const ColumnPtr & column) { size_t size = column->size(); @@ -64,17 +64,17 @@ void CMSketchStatistics::update(const ColumnPtr & column) } } -void CMSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'cmsketch' does not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'countminsketch' does not support type {}", data_type->getName()); } -StatisticsPtr CMSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(stat, data_type); } } diff --git a/src/Storages/Statistics/CountMinSketchStatistics.h b/src/Storages/Statistics/CountMinSketchStatistics.h new file mode 100644 index 00000000000..23ea2cf25c9 --- /dev/null +++ b/src/Storages/Statistics/CountMinSketchStatistics.h @@ -0,0 +1,39 @@ +#pragma once + +#if USE_DATASKETCHES + +#include +#include +#include + +namespace DB +{ + +/// CountMinSketchStatistics is used to estimate expression like col = 'value' or col in ('v1', 'v2'). +class CountMinSketchStatistics : public IStatistics +{ +public: + explicit CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); + + Float64 estimateEqual(const Field & value) const; + + void serialize(WriteBuffer & buf) override; + + void deserialize(ReadBuffer & buf) override; + + void update(const ColumnPtr & column) override; + +private: + static constexpr size_t HASH_COUNT = 8; + static constexpr size_t BUCKET_COUNT = 2048; + + datasketches::count_min_sketch data; + DataTypePtr data_type; +}; + +StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); + +} + +#endif diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index b35d653b7ec..2404d234d7a 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,15 +1,15 @@ #include #include -#include -#include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include #include namespace DB @@ -92,10 +92,10 @@ Float64 ColumnStatistics::estimateEqual(Field val) const } } #if USE_DATASKETCHES - if (stats.contains(StatisticsType::CMSketch)) + if (stats.contains(StatisticsType::CountMinSketch)) { - auto cmsketch_static = std::static_pointer_cast(stats.at(StatisticsType::CMSketch)); - return cmsketch_static->estimateEqual(val); + auto count_min_sketch_static = std::static_pointer_cast(stats.at(StatisticsType::CountMinSketch)); + return count_min_sketch_static->estimateEqual(val); } #endif if (val < - ConditionSelectivityEstimator::threshold || val > ConditionSelectivityEstimator::threshold) @@ -177,8 +177,8 @@ MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() registerValidator(StatisticsType::TDigest, TDigestValidator); registerValidator(StatisticsType::Uniq, UniqValidator); #if USE_DATASKETCHES - registerCreator(StatisticsType::CMSketch, CMSketchCreator); - registerValidator(StatisticsType::CMSketch, CMSketchValidator); + registerCreator(StatisticsType::CountMinSketch, CountMinSketchCreator); + registerValidator(StatisticsType::CountMinSketch, CountMinSketchValidator); #endif } diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 08c79043ac4..23339ca3cfe 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -54,8 +54,8 @@ static StatisticsType stringToStatisticsType(String type) return StatisticsType::TDigest; if (type == "uniq") return StatisticsType::Uniq; - if (type == "cmsketch") - return StatisticsType::CMSketch; + if (type == "countminsketch") + return StatisticsType::CountMinSketch; throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); } @@ -67,8 +67,8 @@ String SingleStatisticsDescription::getTypeName() const return "TDigest"; case StatisticsType::Uniq: return "Uniq"; - case StatisticsType::CMSketch: - return "CMSketch"; + case StatisticsType::CountMinSketch: + return "CountMinSketch"; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index a2005f59de1..03b8fb0d583 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -13,7 +13,7 @@ enum class StatisticsType : UInt8 { TDigest = 0, Uniq = 1, - CMSketch = 2, + CountMinSketch = 2, Max = 63, }; diff --git a/tests/queries/0_stateless/03174_statistics_cmsketch.reference b/tests/queries/0_stateless/03174_statistics_countminsketch.reference similarity index 80% rename from tests/queries/0_stateless/03174_statistics_cmsketch.reference rename to tests/queries/0_stateless/03174_statistics_countminsketch.reference index ea77f317a31..3cec7dd7168 100644 --- a/tests/queries/0_stateless/03174_statistics_cmsketch.reference +++ b/tests/queries/0_stateless/03174_statistics_countminsketch.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.t1\n(\n `a` String STATISTICS(cmsketch),\n `b` Int64 STATISTICS(cmsketch),\n `c` UInt64 STATISTICS(cmsketch),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` String STATISTICS(countminsketch),\n `b` Int64 STATISTICS(countminsketch),\n `c` UInt64 STATISTICS(countminsketch),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) diff --git a/tests/queries/0_stateless/03174_statistics_cmsketch.sql b/tests/queries/0_stateless/03174_statistics_countminsketch.sql similarity index 84% rename from tests/queries/0_stateless/03174_statistics_cmsketch.sql rename to tests/queries/0_stateless/03174_statistics_countminsketch.sql index c45d6186fdf..0d9673309b4 100644 --- a/tests/queries/0_stateless/03174_statistics_cmsketch.sql +++ b/tests/queries/0_stateless/03174_statistics_countminsketch.sql @@ -6,9 +6,9 @@ SET allow_statistics_optimize = 1; CREATE TABLE t1 ( - a String STATISTICS(cmsketch), - b Int64 STATISTICS(cmsketch), - c UInt64 STATISTICS(cmsketch), + a String STATISTICS(countminsketch), + b Int64 STATISTICS(countminsketch), + c UInt64 STATISTICS(countminsketch), pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; @@ -31,9 +31,9 @@ DROP TABLE IF EXISTS t2; SET allow_suspicious_low_cardinality_types=1; CREATE TABLE t2 ( - a LowCardinality(String) STATISTICS(cmsketch), - b Int64 STATISTICS(cmsketch), - c UInt64 STATISTICS(cmsketch), + a LowCardinality(String) STATISTICS(countminsketch), + b Int64 STATISTICS(countminsketch), + c UInt64 STATISTICS(countminsketch), pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; @@ -48,9 +48,9 @@ DROP TABLE IF EXISTS t3; CREATE TABLE t3 ( - a Nullable(String) STATISTICS(cmsketch), - b Int64 STATISTICS(cmsketch), - c UInt64 STATISTICS(cmsketch), + a Nullable(String) STATISTICS(countminsketch), + b Int64 STATISTICS(countminsketch), + c UInt64 STATISTICS(countminsketch), pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; @@ -64,9 +64,9 @@ DROP TABLE IF EXISTS t4; CREATE TABLE t4 ( - a LowCardinality(Nullable(String)) STATISTICS(cmsketch), - b Int64 STATISTICS(cmsketch), - c UInt64 STATISTICS(cmsketch), + a LowCardinality(Nullable(String)) STATISTICS(countminsketch), + b Int64 STATISTICS(countminsketch), + c UInt64 STATISTICS(countminsketch), pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; From 7a27b10ef5a9a6a86e985d5922b85038d1cb03ee Mon Sep 17 00:00:00 2001 From: AntiTopQuark Date: Mon, 24 Jun 2024 23:39:47 +0800 Subject: [PATCH 03/59] support MinMax hyperrectangle for MergeTreeIndexSet --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 66 ++++++++++++++++++-- src/Storages/MergeTree/MergeTreeIndexSet.h | 11 +++- 2 files changed, 70 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index b11cbf1e034..e5708b67e03 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -44,10 +44,12 @@ MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet( const String & index_name_, const Block & index_sample_block_, size_t max_rows_, - MutableColumns && mutable_columns_) + MutableColumns && mutable_columns_, + std::vector && set_hyperrectangle_) : index_name(index_name_) , max_rows(max_rows_) , block(index_sample_block_.cloneWithColumns(std::move(mutable_columns_))) + , set_hyperrectangle(std::move(set_hyperrectangle_)) { } @@ -85,6 +87,15 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const serialization->serializeBinaryBulkWithMultipleStreams(column, 0, size(), settings, state); serialization->serializeBinaryBulkStateSuffix(settings, state); } + + for (size_t i = 0; i < num_columns; ++i) + { + const DataTypePtr & type = block.getByPosition(i).type; + auto serialization = type->getDefaultSerialization(); + + serialization->serializeBinary(set_hyperrectangle[i].left, ostr, {}); + serialization->serializeBinary(set_hyperrectangle[i].right, ostr, {}); + } } void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) @@ -117,6 +128,25 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd serialization->deserializeBinaryBulkStatePrefix(settings, state, nullptr); serialization->deserializeBinaryBulkWithMultipleStreams(elem.column, rows_to_read, settings, state, nullptr); } + + set_hyperrectangle.clear(); + Field min_val; + Field max_val; + for (size_t i = 0; i < num_columns; ++i) + { + const DataTypePtr & type = block.getByPosition(i).type; + auto serialization = type->getDefaultSerialization(); + + serialization->deserializeBinary(min_val, istr, {}); + serialization->deserializeBinary(max_val, istr, {}); + + // NULL_LAST + if (min_val.isNull()) + min_val = POSITIVE_INFINITY; + if (max_val.isNull()) + max_val = POSITIVE_INFINITY; + set_hyperrectangle.emplace_back(min_val, true, max_val, true); + } } @@ -182,10 +212,29 @@ void MergeTreeIndexAggregatorSet::update(const Block & block, size_t * pos, size if (has_new_data) { + FieldRef field_min; + FieldRef field_max; for (size_t i = 0; i < columns.size(); ++i) { auto filtered_column = block.getByName(index_columns[i]).column->filter(filter, block.rows()); columns[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); + + if (const auto * column_nullable = typeid_cast(filtered_column.get())) + column_nullable->getExtremesNullLast(field_min, field_max); + else + filtered_column->getExtremes(field_min, field_max); + + if (set_hyperrectangle.size() <= i) + { + set_hyperrectangle.emplace_back(field_min, true, field_max, true); + } + else + { + set_hyperrectangle[i].left + = applyVisitor(FieldVisitorAccurateLess(), set_hyperrectangle[i].left, field_min) ? set_hyperrectangle[i].left : field_min; + set_hyperrectangle[i].right + = applyVisitor(FieldVisitorAccurateLess(), set_hyperrectangle[i].right, field_max) ? field_max : set_hyperrectangle[i].right; + } } } @@ -221,7 +270,7 @@ bool MergeTreeIndexAggregatorSet::buildFilter( MergeTreeIndexGranulePtr MergeTreeIndexAggregatorSet::getGranuleAndReset() { - auto granule = std::make_shared(index_name, index_sample_block, max_rows, std::move(columns)); + auto granule = std::make_shared(index_name, index_sample_block, max_rows, std::move(columns), std::move(set_hyperrectangle)); switch (data.type) { @@ -240,15 +289,22 @@ MergeTreeIndexGranulePtr MergeTreeIndexAggregatorSet::getGranuleAndReset() return granule; } +KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) +{ + return KeyCondition{filter_actions_dag, context, index.column_names, index.expression}; +} MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( const String & index_name_, const Block & index_sample_block, size_t max_rows_, const ActionsDAGPtr & filter_dag, - ContextPtr context) + ContextPtr context, + const IndexDescription & index_description) : index_name(index_name_) , max_rows(max_rows_) + , index_data_types(index_description.data_types) + , condition(buildCondition(index_description, filter_dag, context)) { for (const auto & name : index_sample_block.getNames()) if (!key_columns.contains(name)) @@ -302,7 +358,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx if (!column->isNullAt(i) && (column->get64(i) & 1)) return true; - return false; + return condition.checkInHyperrectangle(granule.set_hyperrectangle, index_data_types).can_be_true; } @@ -546,7 +602,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const { - return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context); + return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context, index); } MergeTreeIndexPtr setIndexCreator(const IndexDescription & index) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 6efc2effafd..4fe79cb03c5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -22,7 +22,8 @@ struct MergeTreeIndexGranuleSet final : public IMergeTreeIndexGranule const String & index_name_, const Block & index_sample_block_, size_t max_rows_, - MutableColumns && columns_); + MutableColumns && columns_, + std::vector && set_hyperrectangle_); void serializeBinary(WriteBuffer & ostr) const override; void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; @@ -36,6 +37,7 @@ struct MergeTreeIndexGranuleSet final : public IMergeTreeIndexGranule const size_t max_rows; Block block; + std::vector set_hyperrectangle; }; @@ -73,6 +75,7 @@ private: ClearableSetVariants data; Sizes key_sizes; MutableColumns columns; + std::vector set_hyperrectangle; }; @@ -84,7 +87,8 @@ public: const Block & index_sample_block, size_t max_rows_, const ActionsDAGPtr & filter_dag, - ContextPtr context); + ContextPtr context, + const IndexDescription & index_description); bool alwaysUnknownOrTrue() const override; @@ -119,6 +123,9 @@ private: std::unordered_set key_columns; ExpressionActionsPtr actions; String actions_output_column_name; + + DataTypes index_data_types; + KeyCondition condition; }; From e29c8f9aeefb188efc2052d08f123e45b5b3309b Mon Sep 17 00:00:00 2001 From: AntiTopQuark Date: Mon, 24 Jun 2024 23:39:47 +0800 Subject: [PATCH 04/59] support MinMax hyperrectangle for MergeTreeIndexSet --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 67 ++++++++++++++++++-- src/Storages/MergeTree/MergeTreeIndexSet.h | 11 +++- 2 files changed, 72 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index b11cbf1e034..14ab266ded6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -44,10 +44,12 @@ MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet( const String & index_name_, const Block & index_sample_block_, size_t max_rows_, - MutableColumns && mutable_columns_) + MutableColumns && mutable_columns_, + std::vector && set_hyperrectangle_) : index_name(index_name_) , max_rows(max_rows_) , block(index_sample_block_.cloneWithColumns(std::move(mutable_columns_))) + , set_hyperrectangle(std::move(set_hyperrectangle_)) { } @@ -85,6 +87,15 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const serialization->serializeBinaryBulkWithMultipleStreams(column, 0, size(), settings, state); serialization->serializeBinaryBulkStateSuffix(settings, state); } + + for (size_t i = 0; i < num_columns; ++i) + { + const DataTypePtr & type = block.getByPosition(i).type; + auto serialization = type->getDefaultSerialization(); + + serialization->serializeBinary(set_hyperrectangle[i].left, ostr, {}); + serialization->serializeBinary(set_hyperrectangle[i].right, ostr, {}); + } } void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) @@ -117,6 +128,25 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd serialization->deserializeBinaryBulkStatePrefix(settings, state, nullptr); serialization->deserializeBinaryBulkWithMultipleStreams(elem.column, rows_to_read, settings, state, nullptr); } + + set_hyperrectangle.clear(); + Field min_val; + Field max_val; + for (size_t i = 0; i < num_columns; ++i) + { + const DataTypePtr & type = block.getByPosition(i).type; + auto serialization = type->getDefaultSerialization(); + + serialization->deserializeBinary(min_val, istr, {}); + serialization->deserializeBinary(max_val, istr, {}); + + // NULL_LAST + if (min_val.isNull()) + min_val = POSITIVE_INFINITY; + if (max_val.isNull()) + max_val = POSITIVE_INFINITY; + set_hyperrectangle.emplace_back(min_val, true, max_val, true); + } } @@ -182,10 +212,29 @@ void MergeTreeIndexAggregatorSet::update(const Block & block, size_t * pos, size if (has_new_data) { + FieldRef field_min; + FieldRef field_max; for (size_t i = 0; i < columns.size(); ++i) { auto filtered_column = block.getByName(index_columns[i]).column->filter(filter, block.rows()); columns[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); + + if (const auto * column_nullable = typeid_cast(filtered_column.get())) + column_nullable->getExtremesNullLast(field_min, field_max); + else + filtered_column->getExtremes(field_min, field_max); + + if (set_hyperrectangle.size() <= i) + { + set_hyperrectangle.emplace_back(field_min, true, field_max, true); + } + else + { + set_hyperrectangle[i].left + = applyVisitor(FieldVisitorAccurateLess(), set_hyperrectangle[i].left, field_min) ? set_hyperrectangle[i].left : field_min; + set_hyperrectangle[i].right + = applyVisitor(FieldVisitorAccurateLess(), set_hyperrectangle[i].right, field_max) ? field_max : set_hyperrectangle[i].right; + } } } @@ -221,7 +270,7 @@ bool MergeTreeIndexAggregatorSet::buildFilter( MergeTreeIndexGranulePtr MergeTreeIndexAggregatorSet::getGranuleAndReset() { - auto granule = std::make_shared(index_name, index_sample_block, max_rows, std::move(columns)); + auto granule = std::make_shared(index_name, index_sample_block, max_rows, std::move(columns), std::move(set_hyperrectangle)); switch (data.type) { @@ -240,15 +289,22 @@ MergeTreeIndexGranulePtr MergeTreeIndexAggregatorSet::getGranuleAndReset() return granule; } +KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) +{ + return KeyCondition{filter_actions_dag, context, index.column_names, index.expression}; +} MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( const String & index_name_, const Block & index_sample_block, size_t max_rows_, const ActionsDAGPtr & filter_dag, - ContextPtr context) + ContextPtr context, + const IndexDescription & index_description) : index_name(index_name_) , max_rows(max_rows_) + , index_data_types(index_description.data_types) + , condition(buildCondition(index_description, filter_dag, context)) { for (const auto & name : index_sample_block.getNames()) if (!key_columns.contains(name)) @@ -292,6 +348,9 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx size_t size = granule.size(); if (size == 0 || (max_rows != 0 && size > max_rows)) return true; + + if (!condition.checkInHyperrectangle(granule.set_hyperrectangle, index_data_types).can_be_true) + return false; Block result = granule.block; actions->execute(result); @@ -546,7 +605,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const { - return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context); + return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context, index); } MergeTreeIndexPtr setIndexCreator(const IndexDescription & index) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 6efc2effafd..4fe79cb03c5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -22,7 +22,8 @@ struct MergeTreeIndexGranuleSet final : public IMergeTreeIndexGranule const String & index_name_, const Block & index_sample_block_, size_t max_rows_, - MutableColumns && columns_); + MutableColumns && columns_, + std::vector && set_hyperrectangle_); void serializeBinary(WriteBuffer & ostr) const override; void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; @@ -36,6 +37,7 @@ struct MergeTreeIndexGranuleSet final : public IMergeTreeIndexGranule const size_t max_rows; Block block; + std::vector set_hyperrectangle; }; @@ -73,6 +75,7 @@ private: ClearableSetVariants data; Sizes key_sizes; MutableColumns columns; + std::vector set_hyperrectangle; }; @@ -84,7 +87,8 @@ public: const Block & index_sample_block, size_t max_rows_, const ActionsDAGPtr & filter_dag, - ContextPtr context); + ContextPtr context, + const IndexDescription & index_description); bool alwaysUnknownOrTrue() const override; @@ -119,6 +123,9 @@ private: std::unordered_set key_columns; ExpressionActionsPtr actions; String actions_output_column_name; + + DataTypes index_data_types; + KeyCondition condition; }; From 6c0c6fd9fe452631467fcb4da4bfd764bb487bea Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 25 Jun 2024 14:54:50 +0800 Subject: [PATCH 05/59] Fix error caused by type miss match --- src/Storages/Statistics/Statistics.cpp | 16 ++++++++++++++++ src/Storages/Statistics/Statistics.h | 2 ++ .../03174_statistics_countminsketch.reference | 14 +++++++------- .../03174_statistics_countminsketch.sql | 14 +++++++------- 4 files changed, 32 insertions(+), 14 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 2404d234d7a..7afd8b8cd12 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -49,6 +49,20 @@ std::optional getString(const Field & f) return {}; } +bool checkType(const Field & f) +{ + switch (f.getType()) + { + case Field::Types::Int64: + case Field::Types::UInt64: + case Field::Types::Float64: + case Field::Types::String: + return true; + default: + return false; + } +} + IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) {} ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_) @@ -94,6 +108,8 @@ Float64 ColumnStatistics::estimateEqual(Field val) const #if USE_DATASKETCHES if (stats.contains(StatisticsType::CountMinSketch)) { + if (!checkType(val)) + return rows * ConditionSelectivityEstimator::default_normal_cond_factor; auto count_min_sketch_static = std::static_pointer_cast(stats.at(StatisticsType::CountMinSketch)); return count_min_sketch_static->estimateEqual(val); } diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index f6121d72256..b56a1959ea0 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -103,4 +103,6 @@ private: std::optional getFloat64(const Field & f); std::optional getString(const Field & f); +bool checkType(const Field & f); + } diff --git a/tests/queries/0_stateless/03174_statistics_countminsketch.reference b/tests/queries/0_stateless/03174_statistics_countminsketch.reference index 3cec7dd7168..b5a35c22835 100644 --- a/tests/queries/0_stateless/03174_statistics_countminsketch.reference +++ b/tests/queries/0_stateless/03174_statistics_countminsketch.reference @@ -1,26 +1,26 @@ CREATE TABLE default.t1\n(\n `a` String STATISTICS(countminsketch),\n `b` Int64 STATISTICS(countminsketch),\n `c` UInt64 STATISTICS(countminsketch),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'_String), equals(c, 0), greater(b, 0)) (removed) + Prewhere filter column: and(equals(a, \'0\'), equals(c, 0), greater(b, 0)) (removed) After drop statistics for a Prewhere info Prewhere filter - Prewhere filter column: and(equals(b, 0), equals(c, 0), equals(a, \'0\'_String)) (removed) + Prewhere filter column: and(equals(b, 0), equals(c, 0), equals(a, \'0\')) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(equals(c, 0), equals(a, \'0\'_String), greater(b, 0)) (removed) + Prewhere filter column: and(equals(c, 0), equals(a, \'0\'), greater(b, 0)) (removed) LowCardinality Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) Nullable Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) LowCardinality(Nullable) Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'_String), equals(b, 0), equals(c, 0)) (removed) + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) diff --git a/tests/queries/0_stateless/03174_statistics_countminsketch.sql b/tests/queries/0_stateless/03174_statistics_countminsketch.sql index 0d9673309b4..200c2a4a531 100644 --- a/tests/queries/0_stateless/03174_statistics_countminsketch.sql +++ b/tests/queries/0_stateless/03174_statistics_countminsketch.sql @@ -17,14 +17,14 @@ SHOW CREATE TABLE t1; INSERT INTO t1 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b > 0 and a = '0') 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 and b = 0 and a = '0') 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 and b > 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE t1 DROP STATISTICS a; SELECT 'After drop statistics for a'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b > 0 and a = '0') 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 and b = 0 and a = '0') 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 and b > 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; @@ -40,7 +40,7 @@ SETTINGS min_bytes_for_wide_part = 0; INSERT INTO t2 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; SELECT 'LowCardinality'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t2 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t2 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; DROP TABLE IF EXISTS t2; @@ -57,7 +57,7 @@ SETTINGS min_bytes_for_wide_part = 0; INSERT INTO t3 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; SELECT 'Nullable'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t3 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t3 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; DROP TABLE IF EXISTS t3; DROP TABLE IF EXISTS t4; @@ -73,6 +73,6 @@ SETTINGS min_bytes_for_wide_part = 0; INSERT INTO t4 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; SELECT 'LowCardinality(Nullable)'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t4 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t4 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; DROP TABLE IF EXISTS t4; From 7b2868a1ef6ea32b4bdaac8c13186868a9aec4fe Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 26 Jun 2024 10:09:04 +0800 Subject: [PATCH 06/59] A little code refactoring --- .../ConditionSelectivityEstimator.cpp | 4 +-- .../Statistics/CountMinSketchStatistics.cpp | 31 ++++++++++++++----- .../Statistics/CountMinSketchStatistics.h | 2 ++ src/Storages/Statistics/Statistics.cpp | 27 +++++----------- src/Storages/Statistics/Statistics.h | 7 ++--- 5 files changed, 37 insertions(+), 34 deletions(-) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 73c5c549a5d..437b39cb537 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -37,7 +37,7 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreat Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(Field val, Float64 rows) const { - auto float_val = getFloat64(val); + auto float_val = IStatistics::getFloat64(val); if (part_statistics.empty()) { if (!float_val) @@ -148,7 +148,7 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode else dummy = true; auto [op, val] = extractBinaryOp(node, col); - auto float_val = getFloat64(val); + auto float_val = IStatistics::getFloat64(val); if (op == "equals") { if (dummy) diff --git a/src/Storages/Statistics/CountMinSketchStatistics.cpp b/src/Storages/Statistics/CountMinSketchStatistics.cpp index d0f52e3f6df..53415ff3946 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.cpp +++ b/src/Storages/Statistics/CountMinSketchStatistics.cpp @@ -22,13 +22,27 @@ CountMinSketchStatistics::CountMinSketchStatistics(const SingleStatisticsDescrip Float64 CountMinSketchStatistics::estimateEqual(const Field & value) const { - if (auto float_val = getFloat64(value)) + if (auto float_val = IStatistics::getFloat64(value)) return data.get_estimate(&float_val.value(), 8); - if (auto string_val = getString(value)) + if (auto string_val = IStatistics::getString(value)) return data.get_estimate(string_val->data(), string_val->size()); UNREACHABLE(); } +bool CountMinSketchStatistics::checkType(const Field & f) +{ + switch (f.getType()) + { + case Field::Types::Int64: + case Field::Types::UInt64: + case Field::Types::Float64: + case Field::Types::String: + return true; + default: + return false; + } +} + void CountMinSketchStatistics::serialize(WriteBuffer & buf) { auto bytes = data.serialize(); @@ -55,12 +69,13 @@ void CountMinSketchStatistics::update(const ColumnPtr & column) { Field f; column->get(i, f); - if (f.isNull()) - continue; - if (auto float_val = getFloat64(f)) - data.update(&float_val.value(), 8, 1.0); - else if (auto string_val = getString(f)) - data.update(*string_val, 1.0); + if (checkType(f)) + { + if (auto float_val = IStatistics::getFloat64(f)) + data.update(&float_val.value(), 8, 1.0); + else if (auto string_val = IStatistics::getString(f)) + data.update(*string_val, 1.0); + } } } diff --git a/src/Storages/Statistics/CountMinSketchStatistics.h b/src/Storages/Statistics/CountMinSketchStatistics.h index 23ea2cf25c9..37f0aa48749 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.h +++ b/src/Storages/Statistics/CountMinSketchStatistics.h @@ -23,6 +23,8 @@ public: void update(const ColumnPtr & column) override; + bool checkType(const Field & f); + private: static constexpr size_t HASH_COUNT = 8; static constexpr size_t BUCKET_COUNT = 2048; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 7afd8b8cd12..9b184f14bfb 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -27,7 +27,7 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; -std::optional getFloat64(const Field & f) +std::optional IStatistics::getFloat64(const Field & f) { const auto type = f.getType(); Float64 value; @@ -42,27 +42,13 @@ std::optional getFloat64(const Field & f) return value; } -std::optional getString(const Field & f) +std::optional IStatistics::getString(const Field & f) { if (f.getType() == Field::Types::String) return f.get(); return {}; } -bool checkType(const Field & f) -{ - switch (f.getType()) - { - case Field::Types::Int64: - case Field::Types::UInt64: - case Field::Types::Float64: - case Field::Types::String: - return true; - default: - return false; - } -} - IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) {} ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_) @@ -93,7 +79,7 @@ Float64 ColumnStatistics::estimateGreater(Float64 val) const Float64 ColumnStatistics::estimateEqual(Field val) const { - auto float_val = getFloat64(val); + auto float_val = IStatistics::getFloat64(val); if (float_val && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) { auto uniq_static = std::static_pointer_cast(stats.at(StatisticsType::Uniq)); @@ -108,10 +94,11 @@ Float64 ColumnStatistics::estimateEqual(Field val) const #if USE_DATASKETCHES if (stats.contains(StatisticsType::CountMinSketch)) { - if (!checkType(val)) - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; auto count_min_sketch_static = std::static_pointer_cast(stats.at(StatisticsType::CountMinSketch)); - return count_min_sketch_static->estimateEqual(val); + if (!count_min_sketch_static->checkType(val)) + return rows * ConditionSelectivityEstimator::default_normal_cond_factor; + else + return count_min_sketch_static->estimateEqual(val); } #endif if (val < - ConditionSelectivityEstimator::threshold || val > ConditionSelectivityEstimator::threshold) diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index b56a1959ea0..b6df6e45b70 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -35,6 +35,9 @@ public: virtual void update(const ColumnPtr & column) = 0; + static std::optional getFloat64(const Field & f); + static std::optional getString(const Field & f); + protected: SingleStatisticsDescription stat; }; @@ -101,8 +104,4 @@ private: Validators validators; }; -std::optional getFloat64(const Field & f); -std::optional getString(const Field & f); -bool checkType(const Field & f); - } From cc67efd7898483fe165eb0e72e8e61818027aab0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 26 Jun 2024 12:39:50 +0000 Subject: [PATCH 07/59] Some fixups --- .../mergetree-family/mergetree.md | 11 +++++++--- .../statements/alter/statistics.md | 4 ++-- src/CMakeLists.txt | 3 +-- .../Statistics/CountMinSketchStatistics.cpp | 20 ++++++++++--------- .../Statistics/CountMinSketchStatistics.h | 16 +++++++-------- src/Storages/Statistics/Statistics.cpp | 2 ++ src/Storages/Statistics/Statistics.h | 1 - src/Storages/Statistics/TDigestStatistics.h | 1 - src/Storages/Statistics/UniqStatistics.h | 1 - src/Storages/StatisticsDescription.cpp | 8 ++++---- 10 files changed, 36 insertions(+), 31 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index f0c4e1b0e34..bb681faaf1e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -991,13 +991,18 @@ They can be used for prewhere optimization only if we enable `set allow_statisti #### Available Types of Column Statistics {#available-types-of-column-statistics} + - `TDigest` - Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch. + [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. - `Uniq` - - Estimate the number of distinct values of a column by HyperLogLog. + + [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) sketches which provide an estimation how many distinct values a column contains. + +- `CountMin` + + [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} diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 0d1fa59cf86..6880cef0e5c 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -25,9 +25,9 @@ Also, they are replicated, syncing statistics metadata via ZooKeeper. There is an example adding two statistics types to two columns: ``` -ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq, CountMinSketch; +ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; ``` -:::note +:::note Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4efb6004172..b7138d0700e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -545,8 +545,7 @@ if (TARGET ch_contrib::libpqxx) endif() if (TARGET ch_contrib::datasketches) - target_link_libraries (clickhouse_aggregate_functions PRIVATE ch_contrib::datasketches) - dbms_target_link_libraries(PRIVATE ch_contrib::datasketches) + dbms_target_link_libraries(PUBLIC ch_contrib::datasketches) endif () target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) diff --git a/src/Storages/Statistics/CountMinSketchStatistics.cpp b/src/Storages/Statistics/CountMinSketchStatistics.cpp index 53415ff3946..03c76c9709b 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.cpp +++ b/src/Storages/Statistics/CountMinSketchStatistics.cpp @@ -1,8 +1,8 @@ +#include #include #include #include #include -#include #if USE_DATASKETCHES @@ -16,16 +16,18 @@ extern const int ILLEGAL_STATISTICS; CountMinSketchStatistics::CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_), data(HASH_COUNT, BUCKET_COUNT), data_type(data_type_) + : IStatistics(stat_) + , sketch(HASH_COUNT, BUCKET_COUNT) + , data_type(data_type_) { } Float64 CountMinSketchStatistics::estimateEqual(const Field & value) const { if (auto float_val = IStatistics::getFloat64(value)) - return data.get_estimate(&float_val.value(), 8); + return sketch.get_estimate(&float_val.value(), 8); if (auto string_val = IStatistics::getString(value)) - return data.get_estimate(string_val->data(), string_val->size()); + return sketch.get_estimate(string_val->data(), string_val->size()); UNREACHABLE(); } @@ -45,7 +47,7 @@ bool CountMinSketchStatistics::checkType(const Field & f) void CountMinSketchStatistics::serialize(WriteBuffer & buf) { - auto bytes = data.serialize(); + auto bytes = sketch.serialize(); writeIntBinary(static_cast(bytes.size()), buf); buf.write(reinterpret_cast(bytes.data()), bytes.size()); } @@ -58,7 +60,7 @@ void CountMinSketchStatistics::deserialize(ReadBuffer & buf) s.reserve(size); buf.readStrict(s.data(), size); /// Extra copy can be avoided by implementing count_min_sketch::deserialize with ReadBuffer auto read_sketch = datasketches::count_min_sketch::deserialize(s.data(), size, datasketches::DEFAULT_SEED); - data.merge(read_sketch); + sketch.merge(read_sketch); } void CountMinSketchStatistics::update(const ColumnPtr & column) @@ -72,9 +74,9 @@ void CountMinSketchStatistics::update(const ColumnPtr & column) if (checkType(f)) { if (auto float_val = IStatistics::getFloat64(f)) - data.update(&float_val.value(), 8, 1.0); + sketch.update(&float_val.value(), 8, 1.0); else if (auto string_val = IStatistics::getString(f)) - data.update(*string_val, 1.0); + sketch.update(*string_val, 1.0); } } } @@ -84,7 +86,7 @@ void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'countminsketch' does not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'countmin' does not support type {}", data_type->getName()); } StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) diff --git a/src/Storages/Statistics/CountMinSketchStatistics.h b/src/Storages/Statistics/CountMinSketchStatistics.h index 37f0aa48749..a112de4b3aa 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.h +++ b/src/Storages/Statistics/CountMinSketchStatistics.h @@ -1,24 +1,24 @@ #pragma once +#include + +#include "config.h" + #if USE_DATASKETCHES -#include #include -#include namespace DB { -/// CountMinSketchStatistics is used to estimate expression like col = 'value' or col in ('v1', 'v2'). class CountMinSketchStatistics : public IStatistics { public: - explicit CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); + CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); Float64 estimateEqual(const Field & value) const; void serialize(WriteBuffer & buf) override; - void deserialize(ReadBuffer & buf) override; void update(const ColumnPtr & column) override; @@ -26,10 +26,10 @@ public: bool checkType(const Field & f); private: - static constexpr size_t HASH_COUNT = 8; - static constexpr size_t BUCKET_COUNT = 2048; + static constexpr auto HASH_COUNT = 8uz; + static constexpr auto BUCKET_COUNT = 2048uz; - datasketches::count_min_sketch data; + datasketches::count_min_sketch sketch; DataTypePtr data_type; }; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 9b184f14bfb..36339d7456f 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -12,6 +12,8 @@ #include #include +#include "config.h" /// USE_DATASKETCHES + namespace DB { diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index b6df6e45b70..9cc50a8ad4d 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -30,7 +30,6 @@ public: virtual ~IStatistics() = default; virtual void serialize(WriteBuffer & buf) = 0; - virtual void deserialize(ReadBuffer & buf) = 0; virtual void update(const ColumnPtr & column) = 0; diff --git a/src/Storages/Statistics/TDigestStatistics.h b/src/Storages/Statistics/TDigestStatistics.h index 7c361b8751f..396d371aae7 100644 --- a/src/Storages/Statistics/TDigestStatistics.h +++ b/src/Storages/Statistics/TDigestStatistics.h @@ -18,7 +18,6 @@ public: Float64 estimateEqual(Float64 val) const; void serialize(WriteBuffer & buf) override; - void deserialize(ReadBuffer & buf) override; void update(const ColumnPtr & column) override; diff --git a/src/Storages/Statistics/UniqStatistics.h b/src/Storages/Statistics/UniqStatistics.h index 0d86a6e458a..16bf5858ce1 100644 --- a/src/Storages/Statistics/UniqStatistics.h +++ b/src/Storages/Statistics/UniqStatistics.h @@ -17,7 +17,6 @@ public: UInt64 getCardinality(); void serialize(WriteBuffer & buf) override; - void deserialize(ReadBuffer & buf) override; void update(const ColumnPtr & column) override; diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 23339ca3cfe..8fa3400d3a6 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -54,9 +54,9 @@ static StatisticsType stringToStatisticsType(String type) return StatisticsType::TDigest; if (type == "uniq") return StatisticsType::Uniq; - if (type == "countminsketch") + if (type == "count_min") return StatisticsType::CountMinSketch; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq', and 'count_min'.", type); } String SingleStatisticsDescription::getTypeName() const @@ -68,9 +68,9 @@ String SingleStatisticsDescription::getTypeName() const case StatisticsType::Uniq: return "Uniq"; case StatisticsType::CountMinSketch: - return "CountMinSketch"; + return "count_min"; default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq', and 'count_min'.", type); } } From a2ff2a35f0ebdf0cc3b7341b1f798b4d74d780bf Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 26 Jun 2024 14:15:54 +0000 Subject: [PATCH 08/59] More fixups --- .../Statistics/CountMinSketchStatistics.cpp | 13 +++++++------ src/Storages/Statistics/CountMinSketchStatistics.h | 6 ++++-- src/Storages/Statistics/TDigestStatistics.cpp | 10 +++++----- src/Storages/Statistics/TDigestStatistics.h | 2 +- src/Storages/Statistics/UniqStatistics.h | 2 +- 5 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Storages/Statistics/CountMinSketchStatistics.cpp b/src/Storages/Statistics/CountMinSketchStatistics.cpp index 03c76c9709b..c22aa966a15 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.cpp +++ b/src/Storages/Statistics/CountMinSketchStatistics.cpp @@ -47,7 +47,7 @@ bool CountMinSketchStatistics::checkType(const Field & f) void CountMinSketchStatistics::serialize(WriteBuffer & buf) { - auto bytes = sketch.serialize(); + Sketch::vector_bytes bytes = sketch.serialize(); writeIntBinary(static_cast(bytes.size()), buf); buf.write(reinterpret_cast(bytes.data()), bytes.size()); } @@ -56,11 +56,12 @@ void CountMinSketchStatistics::deserialize(ReadBuffer & buf) { UInt64 size; readIntBinary(size, buf); - String s; - s.reserve(size); - buf.readStrict(s.data(), size); /// Extra copy can be avoided by implementing count_min_sketch::deserialize with ReadBuffer - auto read_sketch = datasketches::count_min_sketch::deserialize(s.data(), size, datasketches::DEFAULT_SEED); - sketch.merge(read_sketch); + + Sketch::vector_bytes bytes; + bytes.reserve(size); + buf.readStrict(reinterpret_cast(bytes.data()), size); + + sketch = datasketches::count_min_sketch::deserialize(bytes.data(), size, datasketches::DEFAULT_SEED); } void CountMinSketchStatistics::update(const ColumnPtr & column) diff --git a/src/Storages/Statistics/CountMinSketchStatistics.h b/src/Storages/Statistics/CountMinSketchStatistics.h index a112de4b3aa..2a4f65196a2 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.h +++ b/src/Storages/Statistics/CountMinSketchStatistics.h @@ -29,12 +29,14 @@ private: static constexpr auto HASH_COUNT = 8uz; static constexpr auto BUCKET_COUNT = 2048uz; - datasketches::count_min_sketch sketch; + using Sketch = datasketches::count_min_sketch; + Sketch sketch; + DataTypePtr data_type; }; -StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/TDigestStatistics.cpp b/src/Storages/Statistics/TDigestStatistics.cpp index aa5662c979d..d390619ea36 100644 --- a/src/Storages/Statistics/TDigestStatistics.cpp +++ b/src/Storages/Statistics/TDigestStatistics.cpp @@ -45,11 +45,6 @@ void TDigestStatistics::update(const ColumnPtr & column) } } -StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) -{ - return std::make_shared(stat); -} - void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); @@ -57,4 +52,9 @@ void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' does not support type {}", data_type->getName()); } +StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +{ + return std::make_shared(stat); +} + } diff --git a/src/Storages/Statistics/TDigestStatistics.h b/src/Storages/Statistics/TDigestStatistics.h index 396d371aae7..2113572ac38 100644 --- a/src/Storages/Statistics/TDigestStatistics.h +++ b/src/Storages/Statistics/TDigestStatistics.h @@ -25,7 +25,7 @@ private: QuantileTDigest data; }; -StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/UniqStatistics.h b/src/Storages/Statistics/UniqStatistics.h index 16bf5858ce1..bf097620a86 100644 --- a/src/Storages/Statistics/UniqStatistics.h +++ b/src/Storages/Statistics/UniqStatistics.h @@ -29,7 +29,7 @@ private: }; -StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); } From 3f192248e86749fd23ab3a0fa8da36ab4ba0543c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 27 Jun 2024 22:10:57 +0800 Subject: [PATCH 09/59] Some fixups from code review. Fix null data_type in ColumnStatisticsDescription. Update total_rows event if part has no statistics --- src/Storages/MergeTree/MergeTreeData.cpp | 14 ++- .../ConditionSelectivityEstimator.h | 1 + .../Statistics/CountMinSketchStatistics.cpp | 38 +++----- .../Statistics/CountMinSketchStatistics.h | 8 +- src/Storages/Statistics/Statistics.cpp | 49 +++++++--- src/Storages/Statistics/Statistics.h | 3 + src/Storages/Statistics/TDigestStatistics.cpp | 14 ++- src/Storages/Statistics/UniqStatistics.cpp | 2 + src/Storages/StatisticsDescription.cpp | 7 +- ...64_statistics_estimate_predicate.reference | 25 +++++ .../02864_statistics_estimate_predicate.sql | 94 +++++++++++++++++++ .../02864_statistics_uniq.reference | 35 ------- .../0_stateless/02864_statistics_uniq.sql | 71 -------------- .../03174_statistics_countminsketch.reference | 26 ----- .../03174_statistics_countminsketch.sql | 78 --------------- 15 files changed, 198 insertions(+), 267 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistics_estimate_predicate.reference create mode 100644 tests/queries/0_stateless/02864_statistics_estimate_predicate.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql delete mode 100644 tests/queries/0_stateless/03174_statistics_countminsketch.reference delete mode 100644 tests/queries/0_stateless/03174_statistics_countminsketch.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e0ea4cdbcd..364076bd802 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -497,8 +497,11 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP { auto stats = part->loadStatistics(); /// TODO: We only have one stats file for every part. - for (const auto & stat : stats) - result.merge(part->info.getPartNameV1(), part->rows_count, stat); + 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); } catch (...) { @@ -513,8 +516,11 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP if (!partition_pruner.canBePruned(*part)) { auto stats = part->loadStatistics(); - for (const auto & stat : stats) - result.merge(part->info.getPartNameV1(), part->rows_count, stat); + if (stats.empty()) + result.addRows(part->rows_count); + else + for (const auto & stat : stats) + result.merge(part->info.getPartNameV1(), part->rows_count, stat); } } catch (...) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index 9bf4940e563..ff6218e7ef1 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -46,6 +46,7 @@ public: 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; } }; } diff --git a/src/Storages/Statistics/CountMinSketchStatistics.cpp b/src/Storages/Statistics/CountMinSketchStatistics.cpp index c22aa966a15..497570bd2d1 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.cpp +++ b/src/Storages/Statistics/CountMinSketchStatistics.cpp @@ -1,6 +1,6 @@ #include -#include #include +#include #include #include @@ -17,7 +17,7 @@ extern const int ILLEGAL_STATISTICS; CountMinSketchStatistics::CountMinSketchStatistics(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) : IStatistics(stat_) - , sketch(HASH_COUNT, BUCKET_COUNT) + , sketch(num_hashes, num_buckets) , data_type(data_type_) { } @@ -31,20 +31,6 @@ Float64 CountMinSketchStatistics::estimateEqual(const Field & value) const UNREACHABLE(); } -bool CountMinSketchStatistics::checkType(const Field & f) -{ - switch (f.getType()) - { - case Field::Types::Int64: - case Field::Types::UInt64: - case Field::Types::Float64: - case Field::Types::String: - return true; - default: - return false; - } -} - void CountMinSketchStatistics::serialize(WriteBuffer & buf) { Sketch::vector_bytes bytes = sketch.serialize(); @@ -61,24 +47,24 @@ void CountMinSketchStatistics::deserialize(ReadBuffer & buf) bytes.reserve(size); buf.readStrict(reinterpret_cast(bytes.data()), size); - sketch = datasketches::count_min_sketch::deserialize(bytes.data(), size, datasketches::DEFAULT_SEED); + sketch = datasketches::count_min_sketch::deserialize(bytes.data(), size); } void CountMinSketchStatistics::update(const ColumnPtr & column) { size_t size = column->size(); - for (size_t i = 0; i < size; ++i) { Field f; column->get(i, f); - if (checkType(f)) - { - if (auto float_val = IStatistics::getFloat64(f)) - sketch.update(&float_val.value(), 8, 1.0); - else if (auto string_val = IStatistics::getString(f)) - sketch.update(*string_val, 1.0); - } + + if (f.isNull()) + continue; + + if (auto float_val = IStatistics::getFloat64(f)) + sketch.update(&float_val, 8, 1); + else + sketch.update(f.get(), 1); } } @@ -87,7 +73,7 @@ void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'countmin' does not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) diff --git a/src/Storages/Statistics/CountMinSketchStatistics.h b/src/Storages/Statistics/CountMinSketchStatistics.h index 2a4f65196a2..52046a19b64 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.h +++ b/src/Storages/Statistics/CountMinSketchStatistics.h @@ -23,13 +23,11 @@ public: void update(const ColumnPtr & column) override; - bool checkType(const Field & f); - private: - static constexpr auto HASH_COUNT = 8uz; - static constexpr auto BUCKET_COUNT = 2048uz; + static constexpr auto num_hashes = 8uz; + static constexpr auto num_buckets = 2048uz; - using Sketch = datasketches::count_min_sketch; + using Sketch = datasketches::count_min_sketch; Sketch sketch; DataTypePtr data_type; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 36339d7456f..9db0ccf1023 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -31,17 +31,39 @@ enum StatisticsFileVersion : UInt16 std::optional IStatistics::getFloat64(const Field & f) { - const auto type = f.getType(); - Float64 value; - if (type == Field::Types::Int64) - value = f.get(); - else if (type == Field::Types::UInt64) - value = f.get(); - else if (type == Field::Types::Float64) - value = f.get(); - else - return {}; - return value; + switch (f.getType()) + { + case Field::Types::Bool: + return f.get(); + case Field::Types::Int64: + return f.get(); + case Field::Types::UInt64: + return f.get(); + case Field::Types::Float64: + return f.get(); + case Field::Types::Int128: + return f.get(); + case Field::Types::UInt128: + return f.get(); + case Field::Types::Int256: + return f.get(); + case Field::Types::UInt256: + return f.get(); + case Field::Types::Decimal32: + return f.get().getValue().value; + case Field::Types::Decimal64: + return f.get().getValue().value; + case Field::Types::Decimal128: + return f.get().getValue().value; + case Field::Types::Decimal256: + return f.get().getValue().value; + case Field::Types::IPv4: + return f.get().toUnderType(); + case Field::Types::IPv6: + return f.get().toUnderType(); + default: + return {}; + } } std::optional IStatistics::getString(const Field & f) @@ -97,10 +119,7 @@ Float64 ColumnStatistics::estimateEqual(Field val) const if (stats.contains(StatisticsType::CountMinSketch)) { auto count_min_sketch_static = std::static_pointer_cast(stats.at(StatisticsType::CountMinSketch)); - if (!count_min_sketch_static->checkType(val)) - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; - else - return count_min_sketch_static->estimateEqual(val); + return count_min_sketch_static->estimateEqual(val); } #endif if (val < - ConditionSelectivityEstimator::threshold || val > ConditionSelectivityEstimator::threshold) diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 9cc50a8ad4d..acea6d3435d 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -34,6 +34,9 @@ public: virtual void update(const ColumnPtr & column) = 0; + /// Convert filed to Float64, used when estimating the number of rows. + /// Return a Float64 value if f can be represented by number, otherwise return null. + /// See IDataType::isValueRepresentedByNumber static std::optional getFloat64(const Field & f); static std::optional getString(const Field & f); diff --git a/src/Storages/Statistics/TDigestStatistics.cpp b/src/Storages/Statistics/TDigestStatistics.cpp index d390619ea36..7156ca8e84c 100644 --- a/src/Storages/Statistics/TDigestStatistics.cpp +++ b/src/Storages/Statistics/TDigestStatistics.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -36,18 +37,23 @@ void TDigestStatistics::deserialize(ReadBuffer & buf) void TDigestStatistics::update(const ColumnPtr & column) { size_t size = column->size(); - for (size_t i = 0; i < size; ++i) { - /// TODO: support more types. - Float64 value = column->getFloat64(i); - data.add(value, 1); + Field f; + column->get(i, f); + + if (f.isNull()) + continue; + + if (auto float_val = IStatistics::getFloat64(f)) + data.add(*float_val, 1); } } 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' does not support type {}", data_type->getName()); } diff --git a/src/Storages/Statistics/UniqStatistics.cpp b/src/Storages/Statistics/UniqStatistics.cpp index fc748e769ca..9e65a2fee15 100644 --- a/src/Storages/Statistics/UniqStatistics.cpp +++ b/src/Storages/Statistics/UniqStatistics.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -54,6 +55,7 @@ void UniqStatistics::update(const ColumnPtr & column) void UniqValidator(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 'uniq' does not support type {}", data_type->getName()); } diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 8fa3400d3a6..d59ffb5c8e9 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -103,10 +103,9 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe chassert(merging_column_type); if (column_name.empty()) - { column_name = merging_column_name; - data_type = merging_column_type; - } + + data_type = merging_column_type; for (const auto & [stats_type, stats_desc]: other.types_to_desc) { @@ -125,6 +124,7 @@ void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & oth throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", column_name, other.column_name); types_to_desc = other.types_to_desc; + data_type = other.data_type; } void ColumnStatisticsDescription::clear() @@ -163,6 +163,7 @@ std::vector ColumnStatisticsDescription::fromAST(co const auto & column = columns.getPhysical(physical_column_name); stats.column_name = column.name; + stats.data_type = column.type; stats.types_to_desc = statistics_types; result.push_back(stats); } diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference new file mode 100644 index 00000000000..7215d5fef58 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference @@ -0,0 +1,25 @@ +CREATE TABLE default.t1\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +Test statistics TDigest: + Prewhere info + Prewhere filter + Prewhere filter column: and(less(c, -98), greater(b, 0)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(c, -98), equals(b, 0)) (removed) +Test statistics Uniq: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(b, 0), equals(c, 0)) (removed) +Test statistics count_min: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics multi-types: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900_UInt16)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) +Test LowCardinality and Nullable data type: +t2 diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql new file mode 100644 index 00000000000..a608f18a354 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql @@ -0,0 +1,94 @@ +-- Tags: no-fasttest +DROP TABLE IF EXISTS t1 SYNC; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET allow_suspicious_low_cardinality_types=1; +SET mutations_sync = 2; + +CREATE TABLE t1 +( + a String, + b UInt64, + c Int64, + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'Test statistics TDigest:'; + +ALTER TABLE t1 ADD STATISTICS b, c TYPE tdigest; +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%'; + +ALTER TABLE t1 DROP STATISTICS b, c; + + +SELECT 'Test statistics Uniq:'; + +ALTER TABLE t1 ADD STATISTICS b TYPE uniq, tdigest; +ALTER TABLE t1 MATERIALIZE STATISTICS b; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0/*1000*/ and b = 0/*10*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE t1 DROP STATISTICS b; + + +SELECT 'Test statistics count_min:'; + +ALTER TABLE t1 ADD STATISTICS a TYPE count_min; +ALTER TABLE t1 ADD STATISTICS b TYPE count_min; +ALTER TABLE t1 ADD STATISTICS c TYPE count_min; +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 = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE t1 DROP STATISTICS a, b, c; + + +SELECT 'Test statistics multi-types:'; + +ALTER TABLE t1 ADD STATISTICS a TYPE count_min; +ALTER TABLE t1 ADD STATISTICS b TYPE count_min, uniq, tdigest; +ALTER TABLE t1 ADD STATISTICS c TYPE count_min, uniq, tdigest; +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%'; + +ALTER TABLE t1 DROP STATISTICS a, b, c; + +DROP TABLE IF EXISTS t1 SYNC; + + +SELECT 'Test LowCardinality and Nullable data type:'; +DROP TABLE IF EXISTS t2 SYNC; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE t2 +( + a LowCardinality(Int64) STATISTICS(uniq, tdigest, count_min), + b Nullable(Int64) STATISTICS(uniq, tdigest, count_min), + c LowCardinality(Nullable(Int64)) STATISTICS(uniq, tdigest, count_min), + pk String, +) Engine = MergeTree() ORDER BY pk; + +select table from system.tables where name = 't2'; + +DROP TABLE IF EXISTS t2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference deleted file mode 100644 index 77786dbdd8c..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.reference +++ /dev/null @@ -1,35 +0,0 @@ -CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After modify TDigest - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) -After drop - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql deleted file mode 100644 index c6b51d2a377..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.sql +++ /dev/null @@ -1,71 +0,0 @@ -DROP TABLE IF EXISTS t1; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -CREATE TABLE t1 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Int64 STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE t1; - -INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; -INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -OPTIMIZE TABLE t1 FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT 'After modify TDigest'; -ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; -ALTER TABLE t1 MATERIALIZE STATISTICS c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - - -ALTER TABLE t1 DROP STATISTICS c; - -SELECT 'After drop'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c LowCardinality(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t2; -DROP TABLE IF EXISTS t3; - -CREATE TABLE t3 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Nullable(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t3; diff --git a/tests/queries/0_stateless/03174_statistics_countminsketch.reference b/tests/queries/0_stateless/03174_statistics_countminsketch.reference deleted file mode 100644 index b5a35c22835..00000000000 --- a/tests/queries/0_stateless/03174_statistics_countminsketch.reference +++ /dev/null @@ -1,26 +0,0 @@ -CREATE TABLE default.t1\n(\n `a` String STATISTICS(countminsketch),\n `b` Int64 STATISTICS(countminsketch),\n `c` UInt64 STATISTICS(countminsketch),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(c, 0), greater(b, 0)) (removed) -After drop statistics for a - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(b, 0), equals(c, 0), equals(a, \'0\')) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 0), equals(a, \'0\'), greater(b, 0)) (removed) -LowCardinality - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Nullable - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -LowCardinality(Nullable) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) diff --git a/tests/queries/0_stateless/03174_statistics_countminsketch.sql b/tests/queries/0_stateless/03174_statistics_countminsketch.sql deleted file mode 100644 index 200c2a4a531..00000000000 --- a/tests/queries/0_stateless/03174_statistics_countminsketch.sql +++ /dev/null @@ -1,78 +0,0 @@ --- Tags: no-fasttest -DROP TABLE IF EXISTS t1; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -CREATE TABLE t1 -( - a String STATISTICS(countminsketch), - b Int64 STATISTICS(countminsketch), - c UInt64 STATISTICS(countminsketch), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE t1; - -INSERT INTO t1 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b = 0 and a = '0') 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 and b > 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE t1 DROP STATISTICS a; - -SELECT 'After drop statistics for a'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0 and b = 0 and a = '0') 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 and b > 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 -( - a LowCardinality(String) STATISTICS(countminsketch), - b Int64 STATISTICS(countminsketch), - c UInt64 STATISTICS(countminsketch), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t2 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'LowCardinality'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t2 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - - -DROP TABLE IF EXISTS t2; -DROP TABLE IF EXISTS t3; - -CREATE TABLE t3 -( - a Nullable(String) STATISTICS(countminsketch), - b Int64 STATISTICS(countminsketch), - c UInt64 STATISTICS(countminsketch), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t3 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'Nullable'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t3 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t3; -DROP TABLE IF EXISTS t4; - -CREATE TABLE t4 -( - a LowCardinality(Nullable(String)) STATISTICS(countminsketch), - b Int64 STATISTICS(countminsketch), - c UInt64 STATISTICS(countminsketch), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t4 select toString(number % 1000), number % 100, number % 10, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'LowCardinality(Nullable)'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t4 WHERE c = 0 and b = 0 and a = '0') WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t4; From 1e54b213850bdd34fc1c251a531da04ae9cb03aa Mon Sep 17 00:00:00 2001 From: AntiTopQuark Date: Fri, 28 Jun 2024 22:43:26 +0800 Subject: [PATCH 10/59] fix fast test for MergeTreeIndexSet --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index c2a574beb90..36844648ac7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -361,7 +361,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx if (!column->isNullAt(i) && (column->get64(i) & 1)) return true; - return condition.checkInHyperrectangle(granule.set_hyperrectangle, index_data_types).can_be_true; + return false; } From 9036ce9725c6d273e84d44b862cdecf31fe36d9c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 4 Jul 2024 11:09:47 +0800 Subject: [PATCH 11/59] Some fixups after merging --- .../mergetree-family/mergetree.md | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 16 ++++------ .../ConditionSelectivityEstimator.cpp | 11 ++----- .../ConditionSelectivityEstimator.h | 22 +++++++------- src/Storages/Statistics/Statistics.cpp | 29 +++++++++---------- src/Storages/Statistics/Statistics.h | 5 ++-- ...stics.cpp => StatisticsCountMinSketch.cpp} | 20 +++++++------ ...tatistics.h => StatisticsCountMinSketch.h} | 9 ++---- src/Storages/Statistics/StatisticsTDigest.cpp | 18 ++++++++---- src/Storages/Statistics/StatisticsTDigest.h | 2 +- src/Storages/Statistics/TDigestStatistics.cpp | 0 src/Storages/StatisticsDescription.cpp | 5 ---- .../02864_statistics_estimate_predicate.sql | 2 ++ 13 files changed, 65 insertions(+), 76 deletions(-) rename src/Storages/Statistics/{CountMinSketchStatistics.cpp => StatisticsCountMinSketch.cpp} (77%) rename src/Storages/Statistics/{CountMinSketchStatistics.h => StatisticsCountMinSketch.h} (71%) delete mode 100644 src/Storages/Statistics/TDigestStatistics.cpp diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 6466aebfaa3..7ffbd9a5bae 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -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} diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6f65f65a4a0..edcd443910e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -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 (...) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 437b39cb537..1755f0eb4df 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -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); } diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index ff6218e7ef1..b9127fcd5bf 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -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 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 part_names; std::map column_estimators; - std::pair 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; } }; } diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index d3776a2d38c..cd94ed716cd 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,12 +1,12 @@ -#include -#include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -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(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; diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 4b953ddb364..c9bf3ca4847 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -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; diff --git a/src/Storages/Statistics/CountMinSketchStatistics.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp similarity index 77% rename from src/Storages/Statistics/CountMinSketchStatistics.cpp rename to src/Storages/Statistics/StatisticsCountMinSketch.cpp index 497570bd2d1..dd8ceef4e2d 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -1,8 +1,8 @@ -#include -#include #include +#include #include #include +#include #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(bytes.size()), buf); buf.write(reinterpret_cast(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(bytes.data()), size); - sketch = datasketches::count_min_sketch::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(stat, data_type); + return std::make_shared(stat, data_type); } } diff --git a/src/Storages/Statistics/CountMinSketchStatistics.h b/src/Storages/Statistics/StatisticsCountMinSketch.h similarity index 71% rename from src/Storages/Statistics/CountMinSketchStatistics.h rename to src/Storages/Statistics/StatisticsCountMinSketch.h index 52046a19b64..3ea3f2dbd3b 100644 --- a/src/Storages/Statistics/CountMinSketchStatistics.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -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; Sketch sketch; diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 0747197370c..306338b4ba2 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -1,5 +1,6 @@ #include #include +#include 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()); } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index d3a3bf115ee..8016faac7c6 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -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 t_digest; diff --git a/src/Storages/Statistics/TDigestStatistics.cpp b/src/Storages/Statistics/TDigestStatistics.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index d59ffb5c8e9..8aa954f5eb5 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -1,19 +1,14 @@ #include -#include #include #include #include #include -#include -#include #include #include #include -#include #include -#include namespace DB { diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql index a608f18a354..61bae842631 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql @@ -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%'; From 84853f6d4b6e37160888f46ba61a8a1e70d7583a Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 4 Jul 2024 16:18:11 +0800 Subject: [PATCH 12/59] Fix style checking --- ...64_statistics_estimate_predicate.reference | 2 +- .../02864_statistics_estimate_predicate.sql | 64 ++++++++++--------- .../0_stateless/02864_statistics_uniq.sql | 0 3 files changed, 34 insertions(+), 32 deletions(-) delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference index 7215d5fef58..7c22f308ab9 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference @@ -22,4 +22,4 @@ Test statistics multi-types: Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) Test LowCardinality and Nullable data type: -t2 +tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql index 61bae842631..7fcb85d80f5 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql @@ -1,12 +1,14 @@ -- Tags: no-fasttest -DROP TABLE IF EXISTS t1 SYNC; +-- Tests statistics usages in prewhere optimization. + +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; -CREATE TABLE t1 +CREATE TABLE tab ( a String, b UInt64, @@ -15,75 +17,75 @@ CREATE TABLE t1 ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; -SHOW CREATE TABLE t1; +SHOW CREATE TABLE tab; -INSERT INTO t1 select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; SELECT 'Test statistics TDigest:'; -ALTER TABLE t1 ADD STATISTICS b, c TYPE tdigest; -ALTER TABLE t1 MATERIALIZE STATISTICS b, c; +ALTER TABLE tab ADD STATISTICS b, c TYPE tdigest; +ALTER TABLE tab 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*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab 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*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 0/*1000*/ and c < -98/*100*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE t1 DROP STATISTICS b, c; +ALTER TABLE tab DROP STATISTICS b, c; SELECT 'Test statistics Uniq:'; -ALTER TABLE t1 ADD STATISTICS b TYPE uniq, tdigest; -ALTER TABLE t1 MATERIALIZE STATISTICS b; +ALTER TABLE tab ADD STATISTICS b TYPE uniq, tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS b; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0/*1000*/ and b = 0/*10*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*1000*/ and b = 0/*10*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE t1 DROP STATISTICS b; +ALTER TABLE tab DROP STATISTICS b; SELECT 'Test statistics count_min:'; -ALTER TABLE t1 ADD STATISTICS a TYPE count_min; -ALTER TABLE t1 ADD STATISTICS b TYPE count_min; -ALTER TABLE t1 ADD STATISTICS c TYPE count_min; -ALTER TABLE t1 MATERIALIZE STATISTICS a, b, c; +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab ADD STATISTICS b TYPE count_min; +ALTER TABLE tab ADD STATISTICS c TYPE count_min; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE t1 DROP STATISTICS a, b, c; +ALTER TABLE tab DROP STATISTICS a, b, c; SELECT 'Test statistics multi-types:'; -ALTER TABLE t1 ADD STATISTICS a TYPE count_min; -ALTER TABLE t1 ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE t1 ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE t1 MATERIALIZE STATISTICS a, b, c; +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; +ALTER TABLE tab 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*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab 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*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE t1 DROP STATISTICS a, b, c; +ALTER TABLE tab DROP STATISTICS a, b, c; -DROP TABLE IF EXISTS t1 SYNC; +DROP TABLE IF EXISTS tab SYNC; SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS t2 SYNC; +DROP TABLE IF EXISTS tab2 SYNC; SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 +CREATE TABLE tab2 ( a LowCardinality(Int64) STATISTICS(uniq, tdigest, count_min), b Nullable(Int64) STATISTICS(uniq, tdigest, count_min), @@ -91,6 +93,6 @@ CREATE TABLE t2 pk String, ) Engine = MergeTree() ORDER BY pk; -select table from system.tables where name = 't2'; +select name from system.tables where name = 'tab2' and database = currentDatabase(); -DROP TABLE IF EXISTS t2 SYNC; +DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql deleted file mode 100644 index e69de29bb2d..00000000000 From 943a8bce304c5610ea69a6aea7fdf354c5f71944 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 5 Jul 2024 15:41:21 +0800 Subject: [PATCH 13/59] Estimate should be based on column data type not only constant value --- .../ConditionSelectivityEstimator.cpp | 12 +-- .../ConditionSelectivityEstimator.h | 4 +- src/Storages/Statistics/Statistics.cpp | 38 ++-------- src/Storages/Statistics/Statistics.h | 20 ++--- .../Statistics/StatisticsCountMinSketch.cpp | 76 ++++++++++++------- .../Statistics/StatisticsCountMinSketch.h | 6 +- src/Storages/Statistics/StatisticsTDigest.cpp | 25 +++--- src/Storages/Statistics/StatisticsTDigest.h | 2 +- src/Storages/StatisticsDescription.cpp | 4 +- ...64_statistics_estimate_predicate.reference | 2 +- 10 files changed, 100 insertions(+), 89 deletions(-) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 1755f0eb4df..19e3157d99a 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -16,7 +16,7 @@ void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String par part_statistics[part_name] = stats; } -Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(Float64 val, Float64 rows) const +Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(const Field & val, Float64 rows) const { if (part_statistics.empty()) return default_normal_cond_factor * rows; @@ -30,14 +30,14 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess( return result * rows / part_rows; } -Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreater(Float64 val, Float64 rows) const +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 { - auto float_val = IStatistics::getFloat64(val); + auto float_val = StatisticsUtils::tryConvertToFloat64(val); if (part_statistics.empty()) { if (!float_val) @@ -148,7 +148,7 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode else dummy = true; auto [op, val] = extractBinaryOp(node, col); - auto float_val = IStatistics::getFloat64(val); + auto float_val = StatisticsUtils::tryConvertToFloat64(val); if (op == "equals") { if (dummy) @@ -164,13 +164,13 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode { if (dummy) return default_normal_cond_factor * total_rows; - return estimator.estimateLess(float_val.value(), total_rows); + return estimator.estimateLess(val, total_rows); } else if (op == "greater" || op == "greaterOrEquals") { if (dummy) return default_normal_cond_factor * total_rows; - return estimator.estimateGreater(float_val.value(), total_rows); + return estimator.estimateGreater(val, total_rows); } else return default_unknown_cond_factor * total_rows; diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index b9127fcd5bf..ce7fdd12e92 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -29,9 +29,9 @@ private: void merge(String part_name, ColumnStatisticsPtr stats); - Float64 estimateLess(Float64 val, Float64 rows) const; + Float64 estimateLess(const Field & val, Float64 rows) const; - Float64 estimateGreater(Float64 val, Float64 rows) const; + Float64 estimateGreater(const Field & val, Float64 rows) const; Float64 estimateEqual(const Field & val, Float64 rows) const; }; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index cd94ed716cd..5663c55b263 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,8 +1,8 @@ +#include #include #include #include #include -#include #include #include #include @@ -27,44 +27,22 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; -std::optional IStatistics::getFloat64(const Field & f) +std::optional StatisticsUtils::tryConvertToFloat64(const Field & f) { switch (f.getType()) { - case Field::Types::Bool: - return f.get(); case Field::Types::Int64: return f.get(); case Field::Types::UInt64: return f.get(); case Field::Types::Float64: return f.get(); - case Field::Types::Int128: - return f.get(); - case Field::Types::UInt128: - return f.get(); - case Field::Types::Int256: - return f.get(); - case Field::Types::UInt256: - return f.get(); - case Field::Types::Decimal32: - return f.get().getValue().value; - case Field::Types::Decimal64: - return f.get().getValue().value; - case Field::Types::Decimal128: - return f.get().getValue().value; - case Field::Types::Decimal256: - return f.get().getValue().value; - case Field::Types::IPv4: - return f.get().toUnderType(); - case Field::Types::IPv6: - return f.get().toUnderType(); default: return {}; } } -std::optional IStatistics::getString(const Field & f) +std::optional StatisticsUtils::tryConvertToString(const DB::Field & f) { if (f.getType() == Field::Types::String) return f.get(); @@ -98,7 +76,7 @@ Float64 IStatistics::estimateEqual(const Field & /*val*/) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Equality estimation is not implemented for this type of statistics"); } -Float64 IStatistics::estimateLess(Float64 /*val*/) const +Float64 IStatistics::estimateLess(const Field & /*val*/) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Less-than estimation is not implemented for this type of statistics"); } @@ -113,21 +91,21 @@ Float64 IStatistics::estimateLess(Float64 /*val*/) const /// For that reason, all estimation are performed in a central place (here), and we don't simply pass the predicate to the first statistics /// object that supports it natively. -Float64 ColumnStatistics::estimateLess(Float64 val) const +Float64 ColumnStatistics::estimateLess(const Field & val) const { if (stats.contains(StatisticsType::TDigest)) return stats.at(StatisticsType::TDigest)->estimateLess(val); return rows * ConditionSelectivityEstimator::default_normal_cond_factor; } -Float64 ColumnStatistics::estimateGreater(Float64 val) const +Float64 ColumnStatistics::estimateGreater(const Field & val) const { return rows - estimateLess(val); } Float64 ColumnStatistics::estimateEqual(const Field & val) const { - auto float_val = IStatistics::getFloat64(val); + auto float_val = StatisticsUtils::tryConvertToFloat64(val); 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. @@ -254,7 +232,7 @@ ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescri { auto it = creators.find(type); if (it == creators.end()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' 'count_min'", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type); auto stat_ptr = (it->second)(desc, stats.data_type); column_stat->stats[type] = stat_ptr; } diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index c9bf3ca4847..0df5359adfc 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -14,6 +14,14 @@ namespace DB constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; + +///Returns std::nullopt if input Field cannot be converted to a concrete value +struct StatisticsUtils +{ + static std::optional tryConvertToFloat64(const Field & f); + static std::optional tryConvertToString(const Field & f); +}; + /// Statistics describe properties of the values in the column, /// e.g. how many unique values exist, /// what are the N most frequent values, @@ -36,13 +44,7 @@ public: /// Per-value estimations. /// Throws if the statistics object is not able to do a meaningful estimation. 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. - /// Return a Float64 value if f can be represented by number, otherwise return null. - /// See IDataType::isValueRepresentedByNumber - static std::optional getFloat64(const Field & f); - static std::optional getString(const Field & f); + virtual Float64 estimateLess(const Field & val) const; /// summarized cardinality of values < val in the column protected: SingleStatisticsDescription stat; @@ -65,8 +67,8 @@ public: void update(const ColumnPtr & column); - Float64 estimateLess(Float64 val) const; - Float64 estimateGreater(Float64 val) const; + Float64 estimateLess(const Field & val) const; + Float64 estimateGreater(const Field & val) const; Float64 estimateEqual(const Field & val) const; private: diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index dd8ceef4e2d..abd5cf17946 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #if USE_DATASKETCHES @@ -12,10 +14,13 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_STATISTICS; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -static constexpr auto num_hashes = 8uz; -static constexpr auto num_buckets = 2048uz; +/// Constants chosen based on rolling dices, which provides an error tolerance of 0.1% (ε = 0.001) and a confidence level of 99.9% (δ = 0.001). +/// And sketch the size is 152kb. +static constexpr auto num_hashes = 7uz; +static constexpr auto num_buckets = 2718uz; StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) : IStatistics(stat_) @@ -24,13 +29,49 @@ StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescrip { } -Float64 StatisticsCountMinSketch::estimateEqual(const Field & value) const +Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const { - if (auto float_val = IStatistics::getFloat64(value)) - return sketch.get_estimate(&float_val.value(), 8); - if (auto string_val = IStatistics::getString(value)) - return sketch.get_estimate(string_val->data(), string_val->size()); - UNREACHABLE(); + if (data_type->isValueRepresentedByNumber()) + { + /// convertFieldToType will + /// 1. convert string to number, date, datetime, IPv4, Decimal etc + /// 2. do other conversion + /// 3. return null if val larger than the range of data_type + auto val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + /// We will get the proper data type of val_converted, for example, UInt8 for 1, UInt16 for 257. + auto data_type_converted = applyVisitor(FieldToDataType(), val_converted); + DataTypes data_types = {data_type, data_type_converted}; + auto super_type = tryGetLeastSupertype(data_types); + + /// If data_type is UInt8 but val_typed is UInt16, we should return 0. + if (!super_type->equals(*data_type)) + return 0; + + return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); + } + + if (isStringOrFixedString(data_type)) + { + return sketch.get_estimate(val.get()); + } + + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Statistics 'count_min' does not support estimate constant value of type {}", val.getTypeName()); +} + +void StatisticsCountMinSketch::update(const ColumnPtr & column) +{ + size_t size = column->size(); + for (size_t row = 0; row < size; ++row) + { + if (column->isNullAt(row)) + continue; + auto data = column->getDataAt(row); + sketch.update(data.data, data.size, 1); + } } void StatisticsCountMinSketch::serialize(WriteBuffer & buf) @@ -52,28 +93,11 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) sketch = Sketch::deserialize(bytes.data(), size); } -void StatisticsCountMinSketch::update(const ColumnPtr & column) -{ - size_t size = column->size(); - for (size_t i = 0; i < size; ++i) - { - Field f; - column->get(i, f); - - if (f.isNull()) - continue; - - if (auto float_val = IStatistics::getFloat64(f)) - sketch.update(&float_val, 8, 1); - else - sketch.update(f.get(), 1); - } -} - void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); + /// Numeric, String family, IPv4, IPv6, Date family, Enum family are supported. if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index 3ea3f2dbd3b..2cc93d6e75a 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -16,13 +16,13 @@ class StatisticsCountMinSketch : public IStatistics public: StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); - Float64 estimateEqual(const Field & value) const override; + Float64 estimateEqual(const Field & val) const override; + + void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; - void update(const ColumnPtr & column) override; - private: using Sketch = datasketches::count_min_sketch; Sketch sketch; diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 306338b4ba2..c19d0a0328c 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -6,7 +6,8 @@ namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_STATISTICS; +extern const int ILLEGAL_STATISTICS; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; } StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_) @@ -19,14 +20,14 @@ void StatisticsTDigest::update(const ColumnPtr & column) size_t rows = column->size(); for (size_t row = 0; row < rows; ++row) { - Field f; - column->get(row, f); + Field field; + column->get(row, field); - if (f.isNull()) + if (field.isNull()) continue; - if (auto float_val = IStatistics::getFloat64(f)) - t_digest.add(*float_val, 1); + if (auto field_as_float = StatisticsUtils::tryConvertToFloat64(field)) + t_digest.add(*field_as_float, 1); } } @@ -40,14 +41,20 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) t_digest.deserialize(buf); } -Float64 StatisticsTDigest::estimateLess(Float64 val) const +Float64 StatisticsTDigest::estimateLess(const Field & val) const { - return t_digest.getCountLessThan(val); + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); + if (val_as_float) + return t_digest.getCountLessThan(*val_as_float); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Statistics 'tdigest' does not support estimate constant value of type {}", val.getTypeName()); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - return t_digest.getCountEqual(IStatistics::getFloat64(val).value()); + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); + if (val_as_float) + return t_digest.getCountEqual(*val_as_float); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Statistics 'tdigest' does not support estimate constant value of type {}", val.getTypeName()); } void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 8016faac7c6..801d0787eaf 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -16,7 +16,7 @@ public: void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; - Float64 estimateLess(Float64 val) const override; + Float64 estimateLess(const Field & val) const override; Float64 estimateEqual(const Field & val) const override; private: diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 8aa954f5eb5..9c5fd3604b2 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -51,7 +51,7 @@ 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); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq' and 'count_min'.", type); } String SingleStatisticsDescription::getTypeName() const @@ -65,7 +65,7 @@ String SingleStatisticsDescription::getTypeName() const case StatisticsType::CountMinSketch: return "count_min"; 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 'tdigest', 'uniq' and 'count_min'.", type); } } diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference index 7c22f308ab9..83b921af511 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.t1\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 Test statistics TDigest: Prewhere info Prewhere filter From 64de996c03a6d0d685da6d6aa6c401a9ce710d8f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 9 Jul 2024 17:46:44 +0800 Subject: [PATCH 14/59] Fix address sanitizer container-overflow --- src/Storages/Statistics/StatisticsCountMinSketch.cpp | 7 +++---- .../02864_statistics_estimate_predicate.reference | 2 +- .../02864_statistics_estimate_predicate.sql | 12 ++++++------ 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index abd5cf17946..3e91b4052c4 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -33,10 +33,9 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const { if (data_type->isValueRepresentedByNumber()) { - /// convertFieldToType will + /// 'val' maybe number or string, method 'convertFieldToType' will /// 1. convert string to number, date, datetime, IPv4, Decimal etc - /// 2. do other conversion - /// 3. return null if val larger than the range of data_type + /// 2. return null if val larger than the range of data_type auto val_converted = convertFieldToType(val, *data_type); if (val_converted.isNull()) return 0; @@ -87,7 +86,7 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) readIntBinary(size, buf); Sketch::vector_bytes bytes; - bytes.reserve(size); + bytes.resize(size); /// To avoid 'container-overflow' in AddressSanitizer checking buf.readStrict(reinterpret_cast(bytes.data()), size); sketch = Sketch::deserialize(bytes.data(), size); diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference index 83b921af511..4e41c32750f 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference @@ -17,7 +17,7 @@ Test statistics count_min: Test statistics multi-types: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900_UInt16)) (removed) + Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql index 7fcb85d80f5..91b4f2d05cb 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql @@ -26,11 +26,11 @@ SELECT 'Test statistics TDigest:'; ALTER TABLE tab ADD STATISTICS b, c TYPE tdigest; ALTER TABLE tab MATERIALIZE STATISTICS b, c; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*9990*/ and c < -98/*100*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 0/*1000*/ and c < -98/*100*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; @@ -41,7 +41,7 @@ SELECT 'Test statistics Uniq:'; ALTER TABLE tab ADD STATISTICS b TYPE uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS b; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*1000*/ and b = 0/*10*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; @@ -55,7 +55,7 @@ ALTER TABLE tab ADD STATISTICS b TYPE count_min; ALTER TABLE tab ADD STATISTICS c TYPE count_min; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; @@ -69,11 +69,11 @@ ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_String', '') +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab 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', '') +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; From 3ea555524dd08af3181a5d1896cbe518c0d10736 Mon Sep 17 00:00:00 2001 From: AntiTopQuark Date: Thu, 11 Jul 2024 00:20:40 +0800 Subject: [PATCH 15/59] resolve comments --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 41 ++++++-------------- src/Storages/MergeTree/MergeTreeIndexSet.h | 2 - 2 files changed, 11 insertions(+), 32 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 36844648ac7..284d47ef9e7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -87,15 +87,6 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const serialization->serializeBinaryBulkWithMultipleStreams(column, 0, size(), settings, state); serialization->serializeBinaryBulkStateSuffix(settings, state); } - - for (size_t i = 0; i < num_columns; ++i) - { - const DataTypePtr & type = block.getByPosition(i).type; - auto serialization = type->getDefaultSerialization(); - - serialization->serializeBinary(set_hyperrectangle[i].left, ostr, {}); - serialization->serializeBinary(set_hyperrectangle[i].right, ostr, {}); - } } void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) @@ -117,6 +108,10 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; settings.position_independent_encoding = false; + set_hyperrectangle.clear(); + Field min_val; + Field max_val; + for (size_t i = 0; i < num_columns; ++i) { auto & elem = block.getByPosition(i); @@ -127,24 +122,12 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd serialization->deserializeBinaryBulkStatePrefix(settings, state, nullptr); serialization->deserializeBinaryBulkWithMultipleStreams(elem.column, rows_to_read, settings, state, nullptr); - } - set_hyperrectangle.clear(); - Field min_val; - Field max_val; - for (size_t i = 0; i < num_columns; ++i) - { - const DataTypePtr & type = block.getByPosition(i).type; - auto serialization = type->getDefaultSerialization(); + if (const auto * column_nullable = typeid_cast(elem.column.get())) + column_nullable->getExtremesNullLast(min_val, max_val); + else + elem.column->getExtremes(min_val, max_val); - serialization->deserializeBinary(min_val, istr, {}); - serialization->deserializeBinary(max_val, istr, {}); - - // NULL_LAST - if (min_val.isNull()) - min_val = POSITIVE_INFINITY; - if (max_val.isNull()) - max_val = POSITIVE_INFINITY; set_hyperrectangle.emplace_back(min_val, true, max_val, true); } } @@ -295,18 +278,16 @@ KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr } MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( - const String & index_name_, - const Block & index_sample_block, size_t max_rows_, const ActionsDAGPtr & filter_dag, ContextPtr context, const IndexDescription & index_description) - : index_name(index_name_) + : index_name(index_description.name) , max_rows(max_rows_) , index_data_types(index_description.data_types) , condition(buildCondition(index_description, filter_dag, context)) { - for (const auto & name : index_sample_block.getNames()) + for (const auto & name : index_description.sample_block.getNames()) if (!key_columns.contains(name)) key_columns.insert(name); @@ -605,7 +586,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const { - return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context, index); + return std::make_shared(max_rows, filter_actions_dag, context, index); } MergeTreeIndexPtr setIndexCreator(const IndexDescription & index) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 85f6c73149b..168262360fc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -83,8 +83,6 @@ class MergeTreeIndexConditionSet final : public IMergeTreeIndexCondition { public: MergeTreeIndexConditionSet( - const String & index_name_, - const Block & index_sample_block, size_t max_rows_, const ActionsDAGPtr & filter_dag, ContextPtr context, From c41424d197fe5e583100944bf4a2e47216e664c9 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 11 Jul 2024 10:52:55 +0800 Subject: [PATCH 16/59] Throw exception when datasketches is not enabled --- .../Statistics/StatisticsCountMinSketch.cpp | 37 ++++++--- .../Statistics/StatisticsCountMinSketch.h | 4 +- ...64_statistics_estimate_predicate.reference | 12 --- .../02864_statistics_estimate_predicate.sql | 82 +++++++++---------- 4 files changed, 69 insertions(+), 66 deletions(-) diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index 3e91b4052c4..c25372c69ee 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { -extern const int ILLEGAL_STATISTICS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -23,9 +22,7 @@ static constexpr auto num_hashes = 7uz; static constexpr auto num_buckets = 2718uz; StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_) - , sketch(num_hashes, num_buckets) - , data_type(data_type_) + : IStatistics(stat_), sketch(num_hashes, num_buckets), data_type(data_type_) { } @@ -40,7 +37,7 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const if (val_converted.isNull()) return 0; - /// We will get the proper data type of val_converted, for example, UInt8 for 1, UInt16 for 257. + /// We will get the proper data type of val_converted, for example, Int8 for 1, Int16 for 257. auto data_type_converted = applyVisitor(FieldToDataType(), val_converted); DataTypes data_types = {data_type, data_type_converted}; auto super_type = tryGetLeastSupertype(data_types); @@ -58,7 +55,9 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const } throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Statistics 'count_min' does not support estimate constant value of type {}", val.getTypeName()); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Statistics 'count_min' does not support estimate constant value of type {}", + val.getTypeName()); } void StatisticsCountMinSketch::update(const ColumnPtr & column) @@ -92,20 +91,36 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) sketch = Sketch::deserialize(bytes.data(), size); } +} + +#endif + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME; +extern const int ILLEGAL_STATISTICS; +} + void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); - /// Numeric, String family, IPv4, IPv6, Date family, Enum family are supported. + /// Data types of Numeric, String family, IPv4, IPv6, Date family, Enum family are supported. if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { +#if USE_DATASKETCHES return std::make_shared(stat, data_type); -} - -} - +#else + throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "Statistics of type 'count_min' is not supported in this build, to enable it turn on USE_DATASKETCHES when building."); #endif +} + +} diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index 2cc93d6e75a..f4d49d37bca 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -30,9 +30,9 @@ private: DataTypePtr data_type; }; +#endif + void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); } - -#endif diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference index 4e41c32750f..4027aeefe7b 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference @@ -11,15 +11,3 @@ Test statistics Uniq: Prewhere filter Prewhere filter column: and(equals(b, 0), equals(c, 0)) (removed) Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql index 91b4f2d05cb..6647182ab05 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql @@ -55,44 +55,44 @@ ALTER TABLE tab ADD STATISTICS b TYPE count_min; ALTER TABLE tab ADD STATISTICS c TYPE count_min; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - - -SELECT 'Test statistics multi-types:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab 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|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - -DROP TABLE IF EXISTS tab SYNC; - - -SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS tab2 SYNC; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE tab2 -( - a LowCardinality(Int64) STATISTICS(uniq, tdigest, count_min), - b Nullable(Int64) STATISTICS(uniq, tdigest, count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(uniq, tdigest, count_min), - pk String, -) Engine = MergeTree() ORDER BY pk; - -select name from system.tables where name = 'tab2' and database = currentDatabase(); - -DROP TABLE IF EXISTS tab2 SYNC; +-- SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +-- FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +-- WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +-- +-- ALTER TABLE tab DROP STATISTICS a, b, c; +-- +-- +-- SELECT 'Test statistics multi-types:'; +-- +-- ALTER TABLE tab ADD STATISTICS a TYPE count_min; +-- ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; +-- ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; +-- ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; +-- +-- SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +-- FROM (EXPLAIN actions=1 SELECT count(*) FROM tab 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|_UInt16|_String', '') +-- FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) +-- WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +-- +-- ALTER TABLE tab DROP STATISTICS a, b, c; +-- +-- DROP TABLE IF EXISTS tab SYNC; +-- +-- +-- SELECT 'Test LowCardinality and Nullable data type:'; +-- DROP TABLE IF EXISTS tab2 SYNC; +-- SET allow_suspicious_low_cardinality_types=1; +-- CREATE TABLE tab2 +-- ( +-- a LowCardinality(Int64) STATISTICS(uniq, tdigest, count_min), +-- b Nullable(Int64) STATISTICS(uniq, tdigest, count_min), +-- c LowCardinality(Nullable(Int64)) STATISTICS(uniq, tdigest, count_min), +-- pk String, +-- ) Engine = MergeTree() ORDER BY pk; +-- +-- select name from system.tables where name = 'tab2' and database = currentDatabase(); +-- +-- DROP TABLE IF EXISTS tab2 SYNC; From 465442ff7b9e7c7897c8dc754d1f6e4052303257 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 11 Jul 2024 11:19:44 +0800 Subject: [PATCH 17/59] Fix build error --- .../Statistics/StatisticsCountMinSketch.cpp | 14 ++-- .../Statistics/StatisticsCountMinSketch.h | 5 +- ...64_statistics_estimate_predicate.reference | 12 +++ .../02864_statistics_estimate_predicate.sql | 82 +++++++++---------- 4 files changed, 66 insertions(+), 47 deletions(-) diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index c25372c69ee..ff985b06ee3 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -113,14 +113,18 @@ void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } - +#if USE_DATASKETCHES StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { -#if USE_DATASKETCHES return std::make_shared(stat, data_type); -#else - throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "Statistics of type 'count_min' is not supported in this build, to enable it turn on USE_DATASKETCHES when building."); -#endif } +#else +StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription &, DataTypePtr) +{ + throw Exception( + ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, + "Statistics of type 'count_min' is not supported in this build, to enable it turn on USE_DATASKETCHES when building."); +} +#endif } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index f4d49d37bca..d6141f9f73a 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -30,9 +30,12 @@ private: DataTypePtr data_type; }; +} + #endif +namespace DB +{ void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); - } diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference index 4027aeefe7b..4e41c32750f 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference @@ -11,3 +11,15 @@ Test statistics Uniq: Prewhere filter Prewhere filter column: and(equals(b, 0), equals(c, 0)) (removed) Test statistics count_min: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics multi-types: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) +Test LowCardinality and Nullable data type: +tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql index 6647182ab05..91b4f2d05cb 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql +++ b/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql @@ -55,44 +55,44 @@ ALTER TABLE tab ADD STATISTICS b TYPE count_min; ALTER TABLE tab ADD STATISTICS c TYPE count_min; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; --- SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') --- FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx --- WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; --- --- ALTER TABLE tab DROP STATISTICS a, b, c; --- --- --- SELECT 'Test statistics multi-types:'; --- --- ALTER TABLE tab ADD STATISTICS a TYPE count_min; --- ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; --- ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; --- ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; --- --- SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') --- FROM (EXPLAIN actions=1 SELECT count(*) FROM tab 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|_UInt16|_String', '') --- FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) --- WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; --- --- ALTER TABLE tab DROP STATISTICS a, b, c; --- --- DROP TABLE IF EXISTS tab SYNC; --- --- --- SELECT 'Test LowCardinality and Nullable data type:'; --- DROP TABLE IF EXISTS tab2 SYNC; --- SET allow_suspicious_low_cardinality_types=1; --- CREATE TABLE tab2 --- ( --- a LowCardinality(Int64) STATISTICS(uniq, tdigest, count_min), --- b Nullable(Int64) STATISTICS(uniq, tdigest, count_min), --- c LowCardinality(Nullable(Int64)) STATISTICS(uniq, tdigest, count_min), --- pk String, --- ) Engine = MergeTree() ORDER BY pk; --- --- select name from system.tables where name = 'tab2' and database = currentDatabase(); --- --- DROP TABLE IF EXISTS tab2 SYNC; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS a, b, c; + + +SELECT 'Test statistics multi-types:'; + +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab 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|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS a, b, c; + +DROP TABLE IF EXISTS tab SYNC; + + +SELECT 'Test LowCardinality and Nullable data type:'; +DROP TABLE IF EXISTS tab2 SYNC; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE tab2 +( + a LowCardinality(Int64) STATISTICS(uniq, tdigest, count_min), + b Nullable(Int64) STATISTICS(uniq, tdigest, count_min), + c LowCardinality(Nullable(Int64)) STATISTICS(uniq, tdigest, count_min), + pk String, +) Engine = MergeTree() ORDER BY pk; + +select name from system.tables where name = 'tab2' and database = currentDatabase(); + +DROP TABLE IF EXISTS tab2 SYNC; From f0ac0eccb16146303a4b520291c5039d86d700d2 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 12 Jul 2024 16:37:52 +0800 Subject: [PATCH 18/59] Add unit test --- src/Interpreters/convertFieldToType.cpp | 2 +- src/Storages/Statistics/Statistics.cpp | 16 +++--- src/Storages/Statistics/Statistics.h | 6 +-- .../Statistics/StatisticsCountMinSketch.cpp | 44 +++++++-------- src/Storages/Statistics/tests/gtest_stats.cpp | 53 ++++++++++++++++++- 5 files changed, 83 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 184c263dbdb..1a40b780e9a 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -545,7 +545,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID catch (Exception & e) { if (e.code() == ErrorCodes::UNEXPECTED_DATA_AFTER_PARSED_VALUE) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string {} to type {}", src.get(), type.getName()); + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string '{}' to type {}", src.get(), type.getName()); e.addMessage(fmt::format("while converting '{}' to {}", src.get(), type.getName())); throw; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 5663c55b263..5c0e5f178e1 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -27,25 +27,25 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; -std::optional StatisticsUtils::tryConvertToFloat64(const Field & f) +std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) { - switch (f.getType()) + switch (field.getType()) { case Field::Types::Int64: - return f.get(); + return field.get(); case Field::Types::UInt64: - return f.get(); + return field.get(); case Field::Types::Float64: - return f.get(); + return field.get(); default: return {}; } } -std::optional StatisticsUtils::tryConvertToString(const DB::Field & f) +std::optional StatisticsUtils::tryConvertToString(const DB::Field & field) { - if (f.getType() == Field::Types::String) - return f.get(); + if (field.getType() == Field::Types::String) + return field.get(); return {}; } diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 0df5359adfc..33a5cbac4de 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -15,11 +15,11 @@ constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; -///Returns std::nullopt if input Field cannot be converted to a concrete value struct StatisticsUtils { - static std::optional tryConvertToFloat64(const Field & f); - static std::optional tryConvertToString(const Field & f); + /// Returns std::nullopt if input Field cannot be converted to a concrete value + static std::optional tryConvertToFloat64(const Field & field); + static std::optional tryConvertToString(const Field & field); }; /// Statistics describe properties of the values in the column, diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index ff985b06ee3..a3c6ee8a819 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -4,7 +4,7 @@ #include #include #include -#include + #if USE_DATASKETCHES @@ -13,7 +13,7 @@ namespace DB namespace ErrorCodes { -extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int LOGICAL_ERROR; } /// Constants chosen based on rolling dices, which provides an error tolerance of 0.1% (ε = 0.001) and a confidence level of 99.9% (δ = 0.001). @@ -28,36 +28,32 @@ StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescrip Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const { - if (data_type->isValueRepresentedByNumber()) + /// Try to convert field to data_type. Converting string to proper data types such as: number, date, datetime, IPv4, Decimal etc. + /// Return null if val larger than the range of data_type + /// + /// For example: if data_type is Int32: + /// 1. For 1.0, 1, '1', return Field(1) + /// 2. For 1.1, max_value_int64, return null + Field val_converted; + try { - /// 'val' maybe number or string, method 'convertFieldToType' will - /// 1. convert string to number, date, datetime, IPv4, Decimal etc - /// 2. return null if val larger than the range of data_type - auto val_converted = convertFieldToType(val, *data_type); + val_converted = convertFieldToType(val, *data_type); if (val_converted.isNull()) return 0; - - /// We will get the proper data type of val_converted, for example, Int8 for 1, Int16 for 257. - auto data_type_converted = applyVisitor(FieldToDataType(), val_converted); - DataTypes data_types = {data_type, data_type_converted}; - auto super_type = tryGetLeastSupertype(data_types); - - /// If data_type is UInt8 but val_typed is UInt16, we should return 0. - if (!super_type->equals(*data_type)) - return 0; - - return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); } + catch (...) + { + /// If the conversion fails for example, when converting 'not a number' to Int32, return 0 + return 0; + } + + if (data_type->isValueRepresentedByNumber()) + return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); if (isStringOrFixedString(data_type)) - { return sketch.get_estimate(val.get()); - } - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Statistics 'count_min' does not support estimate constant value of type {}", - val.getTypeName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'count_min' does not support estimate data type of {}", data_type->getName()); } void StatisticsCountMinSketch::update(const ColumnPtr & column) diff --git a/src/Storages/Statistics/tests/gtest_stats.cpp b/src/Storages/Statistics/tests/gtest_stats.cpp index c3c14632ba1..d80f64b8b6b 100644 --- a/src/Storages/Statistics/tests/gtest_stats.cpp +++ b/src/Storages/Statistics/tests/gtest_stats.cpp @@ -1,6 +1,10 @@ #include #include +#include +#include + +using namespace DB; TEST(Statistics, TDigestLessThan) { @@ -39,6 +43,51 @@ TEST(Statistics, TDigestLessThan) std::reverse(data.begin(), data.end()); test_less_than(data, {-1, 1e9, 50000.0, 3000.0, 30.0}, {0, 100000, 50000, 3000, 30}, {0, 0, 0.001, 0.001, 0.001}); - - +} + +using Fields = std::vector; + +template +void testConvertFieldToDataType(const DataTypePtr & data_type, const Fields & fields, const T & expected_value, bool convert_failed = false) +{ + for (const auto & field : fields) + { + Field converted_value; + try + { + converted_value = convertFieldToType(field, *data_type); + } + catch(...) + { + /// Just ignore exceptions + } + if (convert_failed) + ASSERT_TRUE(converted_value.isNull()); + else + ASSERT_EQ(converted_value.template get(), expected_value); + } +} + +TEST(Statistics, convertFieldToType) +{ + Fields fields; + + auto data_type_int8 = DataTypeFactory::instance().get("Int8"); + fields = {1, 1.0, "1"}; + testConvertFieldToDataType(data_type_int8, fields, static_cast(1)); + + fields = {256, 1.1, "not a number"}; + testConvertFieldToDataType(data_type_int8, fields, static_cast(1), true); + + auto data_type_float64 = DataTypeFactory::instance().get("Float64"); + fields = {1, 1.0, "1.0"}; + testConvertFieldToDataType(data_type_float64, fields, static_cast(1.0)); + + auto data_type_string = DataTypeFactory::instance().get("String"); + fields = {1, "1"}; + testConvertFieldToDataType(data_type_string, fields, static_cast("1")); + + auto data_type_date = DataTypeFactory::instance().get("Date"); + fields = {"2024-07-12", 19916}; + testConvertFieldToDataType(data_type_date, fields, static_cast(19916)); } From 4e56b66a9f71b9e339e03f941b5016868ccdd337 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jul 2024 23:16:53 +0200 Subject: [PATCH 19/59] Remove unused local variables --- src/Parsers/ExpressionElementParsers.h | 2 +- src/Parsers/ParserDescribeTableQuery.cpp | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 4e3f29bfe0c..0209e785bff 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -9,7 +9,7 @@ namespace DB { -/** The SELECT subquery is in parenthesis. +/** The SELECT subquery, in parentheses. */ class ParserSubquery : public IParserBase { diff --git a/src/Parsers/ParserDescribeTableQuery.cpp b/src/Parsers/ParserDescribeTableQuery.cpp index 92c0cfacd9b..22bbfdb03e1 100644 --- a/src/Parsers/ParserDescribeTableQuery.cpp +++ b/src/Parsers/ParserDescribeTableQuery.cpp @@ -11,15 +11,12 @@ namespace DB { - bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_describe(Keyword::DESCRIBE); ParserKeyword s_desc(Keyword::DESC); ParserKeyword s_table(Keyword::TABLE); ParserKeyword s_settings(Keyword::SETTINGS); - ParserToken s_dot(TokenType::Dot); - ParserIdentifier name_p; ParserSetQuery parser_settings(true); ASTPtr database; @@ -53,5 +50,4 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return true; } - } From c72045710ff22405ff5a6d7441d33d64c598e9e8 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 15 Jul 2024 03:07:23 +0000 Subject: [PATCH 20/59] fix wrong count result when there is non-deterministic function in predicate Signed-off-by: Duc Canh Le --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 23 +++++++++++++------ src/Storages/VirtualColumnUtils.h | 2 +- ..._with_non_deterministic_function.reference | 2 ++ ..._count_with_non_deterministic_function.sql | 4 ++++ 5 files changed, 24 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference create mode 100644 tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 467a5c82141..fe993f1a435 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1141,7 +1141,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( auto metadata_snapshot = getInMemoryMetadataPtr(); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); - auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr, /*strict=*/ true); if (!filter_dag) return {}; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 778c9e13adb..3f16f582d63 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -271,7 +271,8 @@ bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( const ActionsDAG::Node * node, const Block * allowed_inputs, - ActionsDAG::Nodes & additional_nodes) + ActionsDAG::Nodes & additional_nodes, + bool strict) { if (node->type == ActionsDAG::ActionType::FUNCTION) { @@ -280,8 +281,16 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto & node_copy = additional_nodes.emplace_back(*node); node_copy.children.clear(); for (const auto * child : node->children) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, strict)) + { + /// Expression like (now_allowed AND allowed) is not allowed if strict = true. This is important for + /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is + /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply + /// trivial count. + if (strict) + return nullptr; node_copy.children.push_back(child_copy); + } if (node_copy.children.empty()) return nullptr; @@ -307,7 +316,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { auto & node_copy = additional_nodes.emplace_back(*node); for (auto & child : node_copy.children) - if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes); !child) + if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, strict); !child) return nullptr; return &node_copy; @@ -321,7 +330,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto index_hint_dag = index_hint->getActions()->clone(); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag->getOutputs()) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, strict)) atoms.push_back(child_copy); if (!atoms.empty()) @@ -355,13 +364,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) +ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool strict) { if (!predicate) return nullptr; ActionsDAG::Nodes additional_nodes; - const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes); + const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, strict); if (!res) return nullptr; @@ -370,7 +379,7 @@ ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) { - auto dag = splitFilterDagForAllowedInputs(predicate, &block); + auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*strict=*/ false); if (dag) filterBlockWithDAG(dag, block, context); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..48a0e4fc97f 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -32,7 +32,7 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context); bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); /// Extract a part of predicate that can be evaluated using only columns from input_names. -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); +ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool strict = false); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql new file mode 100644 index 00000000000..d4ffa4d07ac --- /dev/null +++ b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql @@ -0,0 +1,4 @@ +CREATE TABLE t (p UInt8, x UInt64) Engine = MergeTree PARTITION BY p ORDER BY x; +INSERT INTO t SELECT 0, number FROM numbers(10); +SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 0; +SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 1; From f10cb5ffb750fb96996bcfdef0f90e396e32fb31 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 15 Jul 2024 12:04:44 +0800 Subject: [PATCH 21/59] Some fixups --- .../ConditionSelectivityEstimator.cpp | 9 ++-- src/Storages/Statistics/Statistics.h | 2 +- .../Statistics/StatisticsCountMinSketch.cpp | 43 ++++--------------- .../Statistics/StatisticsCountMinSketch.h | 10 ++--- src/Storages/Statistics/StatisticsTDigest.cpp | 4 +- src/Storages/Statistics/tests/gtest_stats.cpp | 2 +- 6 files changed, 21 insertions(+), 49 deletions(-) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 19e3157d99a..57dff958b9a 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -37,9 +37,9 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreat Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(const Field & val, Float64 rows) const { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); if (part_statistics.empty()) { + auto float_val = StatisticsUtils::tryConvertToFloat64(val); if (!float_val) return default_unknown_cond_factor * rows; else if (float_val.value() < - threshold || float_val.value() > threshold) @@ -133,9 +133,8 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode { auto result = tryToExtractSingleColumn(node); if (result.second != 1) - { return default_unknown_cond_factor * total_rows; - } + String col = result.first; auto it = column_estimators.find(col); @@ -147,12 +146,14 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode estimator = it->second; else dummy = true; + auto [op, val] = extractBinaryOp(node, col); - auto float_val = StatisticsUtils::tryConvertToFloat64(val); + if (op == "equals") { if (dummy) { + auto float_val = StatisticsUtils::tryConvertToFloat64(val); if (!float_val || (float_val < - threshold || float_val > threshold)) return default_normal_cond_factor * total_rows; else diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 33a5cbac4de..16f0c67eabd 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -1,9 +1,9 @@ #pragma once #include +#include #include #include -#include #include #include diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index a3c6ee8a819..95a8ceda8c8 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -1,11 +1,11 @@ + +#include #include #include #include #include -#include #include - #if USE_DATASKETCHES namespace DB @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; +extern const int ILLEGAL_STATISTICS; } /// Constants chosen based on rolling dices, which provides an error tolerance of 0.1% (ε = 0.001) and a confidence level of 99.9% (δ = 0.001). @@ -34,18 +35,9 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const /// For example: if data_type is Int32: /// 1. For 1.0, 1, '1', return Field(1) /// 2. For 1.1, max_value_int64, return null - Field val_converted; - try - { - val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) - return 0; - } - catch (...) - { - /// If the conversion fails for example, when converting 'not a number' to Int32, return 0 + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) return 0; - } if (data_type->isValueRepresentedByNumber()) return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); @@ -87,19 +79,6 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) sketch = Sketch::deserialize(bytes.data(), size); } -} - -#endif - - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME; -extern const int ILLEGAL_STATISTICS; -} void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { @@ -109,18 +88,12 @@ void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -#if USE_DATASKETCHES + StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } -#else -StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription &, DataTypePtr) -{ - throw Exception( - ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, - "Statistics of type 'count_min' is not supported in this build, to enable it turn on USE_DATASKETCHES when building."); -} -#endif } + +#endif diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index d6141f9f73a..aa71c643c05 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -30,12 +30,10 @@ private: DataTypePtr data_type; }; + +void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); + } #endif - -namespace DB -{ -void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); -} diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index c19d0a0328c..f3c6ce1566b 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -7,7 +7,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_STATISTICS; -extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int LOGICAL_ERROR; } StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_) @@ -54,7 +54,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); if (val_as_float) return t_digest.getCountEqual(*val_as_float); - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Statistics 'tdigest' does not support estimate constant value of type {}", val.getTypeName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); } void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) diff --git a/src/Storages/Statistics/tests/gtest_stats.cpp b/src/Storages/Statistics/tests/gtest_stats.cpp index d80f64b8b6b..f82a535bebc 100644 --- a/src/Storages/Statistics/tests/gtest_stats.cpp +++ b/src/Storages/Statistics/tests/gtest_stats.cpp @@ -59,7 +59,7 @@ void testConvertFieldToDataType(const DataTypePtr & data_type, const Fields & fi } catch(...) { - /// Just ignore exceptions + ASSERT_FALSE(convert_failed); } if (convert_failed) ASSERT_TRUE(converted_value.isNull()); From e8aa157c56e035449b90e3c428f27db0c40263b5 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 15 Jul 2024 14:25:39 +0800 Subject: [PATCH 22/59] Fix code style --- src/Storages/Statistics/StatisticsTDigest.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index f3c6ce1566b..e3a59f3251a 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -46,7 +46,7 @@ Float64 StatisticsTDigest::estimateLess(const Field & val) const auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); if (val_as_float) return t_digest.getCountLessThan(*val_as_float); - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Statistics 'tdigest' does not support estimate constant value of type {}", val.getTypeName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimate value of type {}", val.getTypeName()); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const From 76e1dea7ccdabc27129fdf911845f2622bbbc371 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 15 Jul 2024 18:17:09 +0800 Subject: [PATCH 23/59] Fix unit test --- src/Storages/Statistics/tests/gtest_stats.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Statistics/tests/gtest_stats.cpp b/src/Storages/Statistics/tests/gtest_stats.cpp index f82a535bebc..9b9fae83109 100644 --- a/src/Storages/Statistics/tests/gtest_stats.cpp +++ b/src/Storages/Statistics/tests/gtest_stats.cpp @@ -59,7 +59,7 @@ void testConvertFieldToDataType(const DataTypePtr & data_type, const Fields & fi } catch(...) { - ASSERT_FALSE(convert_failed); + ASSERT_TRUE(convert_failed); } if (convert_failed) ASSERT_TRUE(converted_value.isNull()); From 77d6e781f593fdcfe50506aba855eda747c05c01 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 16 Jul 2024 11:25:18 +0800 Subject: [PATCH 24/59] Update src/Storages/VirtualColumnUtils.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Storages/VirtualColumnUtils.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index f7ffd47f9e8..df833fa6a66 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -282,15 +282,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( node_copy.children.clear(); for (const auto * child : node->children) if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, strict)) - { - /// Expression like (now_allowed AND allowed) is not allowed if strict = true. This is important for - /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is - /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply - /// trivial count. - if (strict) - return nullptr; node_copy.children.push_back(child_copy); - } + /// Expression like (now_allowed AND allowed) is not allowed if strict = true. This is important for + /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is + /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply + /// trivial count. + else if (strict) + return nullptr; if (node_copy.children.empty()) return nullptr; From 85c42348bcb47325be61505a7d908b46be9fe3b3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 16 Jul 2024 11:28:59 +0000 Subject: [PATCH 25/59] address reviews Signed-off-by: Duc Canh Le --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 18 +++++++++--------- src/Storages/VirtualColumnUtils.h | 10 +++++++++- ...3_count_with_non_deterministic_function.sql | 2 +- 4 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index eb07df8f012..faf2741a456 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1144,7 +1144,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( auto metadata_snapshot = getInMemoryMetadataPtr(); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); - auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr, /*strict=*/ true); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr, /*allow_non_deterministic_functions=*/ false); if (!filter_dag) return {}; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index df833fa6a66..151079154b1 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -272,7 +272,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( const ActionsDAG::Node * node, const Block * allowed_inputs, ActionsDAG::Nodes & additional_nodes, - bool strict) + bool allow_non_deterministic_functions) { if (node->type == ActionsDAG::ActionType::FUNCTION) { @@ -281,13 +281,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto & node_copy = additional_nodes.emplace_back(*node); node_copy.children.clear(); for (const auto * child : node->children) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, strict)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) node_copy.children.push_back(child_copy); - /// Expression like (now_allowed AND allowed) is not allowed if strict = true. This is important for + /// Expression like (now_allowed AND allowed) is not allowed if allow_non_deterministic_functions = true. This is important for /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply /// trivial count. - else if (strict) + else if (!allow_non_deterministic_functions) return nullptr; if (node_copy.children.empty()) @@ -314,7 +314,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { auto & node_copy = additional_nodes.emplace_back(*node); for (auto & child : node_copy.children) - if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, strict); !child) + if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions); !child) return nullptr; return &node_copy; @@ -328,7 +328,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto index_hint_dag = index_hint->getActions()->clone(); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag->getOutputs()) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, strict)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) atoms.push_back(child_copy); if (!atoms.empty()) @@ -362,13 +362,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool strict) +ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions) { if (!predicate) return nullptr; ActionsDAG::Nodes additional_nodes; - const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, strict); + const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, allow_non_deterministic_functions); if (!res) return nullptr; @@ -377,7 +377,7 @@ ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) { - auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*strict=*/ false); + auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); if (dag) filterBlockWithDAG(dag, block, context); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 5b41e56c0ef..e5cfa47c8f6 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -32,7 +32,15 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); /// Extract a part of predicate that can be evaluated using only columns from input_names. -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool strict = false); +/// When allow_non_deterministic_functions is true then even if the predicate contains non-deterministic +/// functions, we still allow to extract a part of the predicate, otherwise we return nullptr. +/// allow_non_deterministic_functions must be false when we are going to use the result to filter parts in +/// MergeTreeData::totalRowsByPartitionPredicateImp. For example, if the query is +/// `SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1` +/// The predicate will be `_partition_id = '0' AND rowNumberInBlock() = 1`, and `rowNumberInBlock()` is +/// non-deterministic. If we still extract the part `_partition_id = '0'` for filtering parts, then trivial +/// count optimization will be mistakenly applied to the query. +ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions = true); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql index d4ffa4d07ac..bb3269da597 100644 --- a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql +++ b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql @@ -1,4 +1,4 @@ CREATE TABLE t (p UInt8, x UInt64) Engine = MergeTree PARTITION BY p ORDER BY x; -INSERT INTO t SELECT 0, number FROM numbers(10); +INSERT INTO t SELECT 0, number FROM numbers(10) SETTINGS max_block_size = 100; SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 0; SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 1; From b05e02625844930892d7ceaa77f2655765c51b88 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 17 Jul 2024 09:58:57 +0800 Subject: [PATCH 26/59] Fix logical error --- src/Storages/Statistics/Statistics.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 5c0e5f178e1..588e20e801f 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -37,6 +37,14 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) return field.get(); case Field::Types::Float64: return field.get(); + case Field::Types::Int128: + return field.get(); + case Field::Types::UInt128: + return field.get(); + case Field::Types::Int256: + return field.get(); + case Field::Types::UInt256: + return field.get(); default: return {}; } From b8cf8829927dedc701b97893b3a1e7651193b586 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 17 Jul 2024 23:22:24 +0100 Subject: [PATCH 27/59] impl --- src/Common/CgroupsMemoryUsageObserver.cpp | 49 ++++++++++++++++++----- src/Common/CgroupsMemoryUsageObserver.h | 2 + 2 files changed, 40 insertions(+), 11 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 02bde0d80b7..cf661174789 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -45,26 +46,33 @@ namespace /// kernel 5 /// rss 15 /// [...] -uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) +using Metrics = std::map; + +Metrics readAllMetricsFromStatFile(ReadBufferFromFile & buf) { + Metrics metrics; while (!buf.eof()) { std::string current_key; readStringUntilWhitespace(current_key, buf); - if (current_key != key) - { - std::string dummy; - readStringUntilNewlineInto(dummy, buf); - buf.ignore(); - continue; - } assertChar(' ', buf); + uint64_t value = 0; readIntText(value, buf); - return value; - } + assertChar('\n', buf); + auto [_, inserted] = metrics.emplace(std::move(current_key), value); + chassert(inserted, "Duplicate keys in stat file"); + } + return metrics; +} + +uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) +{ + const auto all_metrics = readAllMetricsFromStatFile(buf); + if (const auto it = all_metrics.find(key); it != all_metrics.end()) + return it->second; throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find '{}' in '{}'", key, buf.getFileName()); } @@ -79,6 +87,13 @@ struct CgroupsV1Reader : ICgroupsReader return readMetricFromStatFile(buf, "rss"); } + std::string dumpAllStats() override + { + std::lock_guard lock(mutex); + buf.rewind(); + return fmt::format("{}", readAllMetricsFromStatFile(buf)); + } + private: std::mutex mutex; ReadBufferFromFile buf TSA_GUARDED_BY(mutex); @@ -106,6 +121,13 @@ struct CgroupsV2Reader : ICgroupsReader return mem_usage; } + std::string dumpAllStats() override + { + std::lock_guard lock(mutex); + stat_buf.rewind(); + return fmt::format("{}", readAllMetricsFromStatFile(stat_buf)); + } + private: std::mutex mutex; ReadBufferFromFile current_buf TSA_GUARDED_BY(mutex); @@ -234,7 +256,12 @@ void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint # endif /// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them. uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); + LOG_TRACE( + log, + "Read current memory usage {} bytes ({}) from cgroups, full available stats: {}", + memory_usage, + ReadableSize(memory_usage), + cgroup_reader->dumpAllStats()); MemoryTracker::setRSS(memory_usage, 0); LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage)); diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index b848a2bff3c..0d5d07597c8 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -14,6 +14,8 @@ struct ICgroupsReader virtual ~ICgroupsReader() = default; virtual uint64_t readMemoryUsage() = 0; + + virtual std::string dumpAllStats() = 0; }; /// Does two things: From 89f6f74418fa361c65261447ebbd0d79d88cc748 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 18 Jul 2024 10:31:11 +0800 Subject: [PATCH 28/59] Revert 02864_statistics_uniq --- .../02864_statistics_uniq.reference | 113 ++++++++++++++++++ .../0_stateless/02864_statistics_uniq.sql | 73 +++++++++++ 2 files changed, 186 insertions(+) create mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference create mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference new file mode 100644 index 00000000000..861e35ddd48 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.reference @@ -0,0 +1,113 @@ +CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After modify TDigest + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) +After drop + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) + 72 changes: 0 additions & 72 deletions72 +tests/queries/0_stateless/02864_statistics_uniq.sql +Viewed +Original file line number Diff line number Diff line change +@@ -1,72 +0,0 @@ +DROP TABLE IF EXISTS t1; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Int64 STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); + +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +OPTIMIZE TABLE t1 FINAL; + +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT 'After modify TDigest'; +ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; +ALTER TABLE t1 MATERIALIZE STATISTICS c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + + +ALTER TABLE t1 DROP STATISTICS c; + +SELECT 'After drop'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE t2 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c LowCardinality(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t3 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Nullable(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t3; + diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql new file mode 100644 index 00000000000..0f5f353c045 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.sql @@ -0,0 +1,73 @@ +DROP TABLE IF EXISTS t1; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Int64 STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); + +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +OPTIMIZE TABLE t1 FINAL; + +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT 'After modify TDigest'; +ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; +ALTER TABLE t1 MATERIALIZE STATISTICS c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + + +ALTER TABLE t1 DROP STATISTICS c; + +SELECT 'After drop'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE t2 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c LowCardinality(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t3 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Nullable(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t3; + From 1c883b802db839220a8e0a81892f653152a59cc1 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 18 Jul 2024 10:33:05 +0800 Subject: [PATCH 29/59] Rename test 02864_statistics_count_min_sketch --- ...864_statistics_count_min_sketch.reference} | 11 --- ... => 02864_statistics_count_min_sketch.sql} | 34 +------- .../02864_statistics_uniq.reference | 78 ------------------- 3 files changed, 3 insertions(+), 120 deletions(-) rename tests/queries/0_stateless/{02864_statistics_estimate_predicate.reference => 02864_statistics_count_min_sketch.reference} (63%) rename tests/queries/0_stateless/{02864_statistics_estimate_predicate.sql => 02864_statistics_count_min_sketch.sql} (62%) diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference similarity index 63% rename from tests/queries/0_stateless/02864_statistics_estimate_predicate.reference rename to tests/queries/0_stateless/02864_statistics_count_min_sketch.reference index 4e41c32750f..02c41656a36 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.reference +++ b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference @@ -1,15 +1,4 @@ CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics TDigest: - Prewhere info - Prewhere filter - Prewhere filter column: and(less(c, -98), greater(b, 0)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(c, -98), equals(b, 0)) (removed) -Test statistics Uniq: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(b, 0), equals(c, 0)) (removed) Test statistics count_min: Prewhere info Prewhere filter diff --git a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql similarity index 62% rename from tests/queries/0_stateless/02864_statistics_estimate_predicate.sql rename to tests/queries/0_stateless/02864_statistics_count_min_sketch.sql index 91b4f2d05cb..c730aa7b4a7 100644 --- a/tests/queries/0_stateless/02864_statistics_estimate_predicate.sql +++ b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql @@ -1,5 +1,4 @@ -- Tags: no-fasttest --- Tests statistics usages in prewhere optimization. DROP TABLE IF EXISTS tab SYNC; @@ -21,33 +20,6 @@ SHOW CREATE TABLE tab; INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; -SELECT 'Test statistics TDigest:'; - -ALTER TABLE tab ADD STATISTICS b, c TYPE tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*9990*/ and c < -98/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 0/*1000*/ and c < -98/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS b, c; - - -SELECT 'Test statistics Uniq:'; - -ALTER TABLE tab ADD STATISTICS b TYPE uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS b; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*1000*/ and b = 0/*10*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS b; - - SELECT 'Test statistics count_min:'; ALTER TABLE tab ADD STATISTICS a TYPE count_min; @@ -87,9 +59,9 @@ DROP TABLE IF EXISTS tab2 SYNC; SET allow_suspicious_low_cardinality_types=1; CREATE TABLE tab2 ( - a LowCardinality(Int64) STATISTICS(uniq, tdigest, count_min), - b Nullable(Int64) STATISTICS(uniq, tdigest, count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(uniq, tdigest, count_min), + a LowCardinality(Int64) STATISTICS(count_min), + b Nullable(Int64) STATISTICS(count_min), + c LowCardinality(Nullable(Int64)) STATISTICS(count_min), pk String, ) Engine = MergeTree() ORDER BY pk; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference index 861e35ddd48..77786dbdd8c 100644 --- a/tests/queries/0_stateless/02864_statistics_uniq.reference +++ b/tests/queries/0_stateless/02864_statistics_uniq.reference @@ -33,81 +33,3 @@ After drop Prewhere info Prewhere filter Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) - 72 changes: 0 additions & 72 deletions72 -tests/queries/0_stateless/02864_statistics_uniq.sql -Viewed -Original file line number Diff line number Diff line change -@@ -1,72 +0,0 @@ -DROP TABLE IF EXISTS t1; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET mutations_sync = 1; - -CREATE TABLE t1 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Int64 STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE t1; - -INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; -INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -OPTIMIZE TABLE t1 FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT 'After modify TDigest'; -ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; -ALTER TABLE t1 MATERIALIZE STATISTICS c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - - -ALTER TABLE t1 DROP STATISTICS c; - -SELECT 'After drop'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c LowCardinality(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t2; -DROP TABLE IF EXISTS t3; - -CREATE TABLE t3 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Nullable(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t3; - From 4eb5a404430f69785fd756d1706621fe5410f43c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 18 Jul 2024 19:04:42 +0100 Subject: [PATCH 30/59] add unit tests --- src/Common/CgroupsMemoryUsageObserver.cpp | 12 +- src/Common/CgroupsMemoryUsageObserver.h | 3 + src/Common/tests/gtest_cgroups_reader.cpp | 175 ++++++++++++++++++++++ 3 files changed, 186 insertions(+), 4 deletions(-) create mode 100644 src/Common/tests/gtest_cgroups_reader.cpp diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index cf661174789..ef8bdfc1823 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -200,10 +200,7 @@ CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait { const auto [cgroup_path, version] = getCgroupsPath(); - if (version == CgroupsVersion::V2) - cgroup_reader = std::make_unique(cgroup_path); - else - cgroup_reader = std::make_unique(cgroup_path); + cgroup_reader = createCgroupsReader(version, cgroup_path); LOG_INFO( log, @@ -365,6 +362,13 @@ void CgroupsMemoryUsageObserver::runThread() } } +std::unique_ptr createCgroupsReader(CgroupsMemoryUsageObserver::CgroupsVersion version, const fs::path & cgroup_path) +{ + if (version == CgroupsMemoryUsageObserver::CgroupsVersion::V2) + return std::make_unique(cgroup_path); + else + return std::make_unique(cgroup_path); +} } #endif diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index 0d5d07597c8..7f888fe631b 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -83,6 +83,9 @@ private: bool quit = false; }; +std::unique_ptr +createCgroupsReader(CgroupsMemoryUsageObserver::CgroupsVersion version, const std::filesystem::path & cgroup_path); + #else class CgroupsMemoryUsageObserver { diff --git a/src/Common/tests/gtest_cgroups_reader.cpp b/src/Common/tests/gtest_cgroups_reader.cpp new file mode 100644 index 00000000000..38e56401401 --- /dev/null +++ b/src/Common/tests/gtest_cgroups_reader.cpp @@ -0,0 +1,175 @@ +#include +#include +#include +#include + +#include +#include +#include +#include "IO/WriteBufferFromFileBase.h" + +using namespace DB; + + +const std::string SAMPLE_FILE[2] = { + R"(cache 4673703936 +rss 2232029184 +rss_huge 0 +shmem 0 +mapped_file 344678400 +dirty 4730880 +writeback 135168 +swap 0 +pgpgin 2038569918 +pgpgout 2036883790 +pgfault 2055373287 +pgmajfault 0 +inactive_anon 2156335104 +active_anon 0 +inactive_file 2841305088 +active_file 1653915648 +unevictable 256008192 +hierarchical_memory_limit 8589934592 +hierarchical_memsw_limit 8589934592 +total_cache 4673703936 +total_rss 2232029184 +total_rss_huge 0 +total_shmem 0 +total_mapped_file 344678400 +total_dirty 4730880 +total_writeback 135168 +total_swap 0 +total_pgpgin 2038569918 +total_pgpgout 2036883790 +total_pgfault 2055373287 +total_pgmajfault 0 +total_inactive_anon 2156335104 +total_active_anon 0 +total_inactive_file 2841305088 +total_active_file 1653915648 +total_unevictable 256008192 +)", + R"(anon 10429399040 +file 17410793472 +kernel 1537789952 +kernel_stack 3833856 +pagetables 65441792 +sec_pagetables 0 +percpu 15232 +sock 0 +vmalloc 0 +shmem 0 +zswap 0 +zswapped 0 +file_mapped 344010752 +file_dirty 2060857344 +file_writeback 0 +swapcached 0 +anon_thp 0 +file_thp 0 +shmem_thp 0 +inactive_anon 0 +active_anon 10429370368 +inactive_file 8693084160 +active_file 8717561856 +unevictable 0 +slab_reclaimable 1460982504 +slab_unreclaimable 5152864 +slab 1466135368 +workingset_refault_anon 0 +workingset_refault_file 0 +workingset_activate_anon 0 +workingset_activate_file 0 +workingset_restore_anon 0 +workingset_restore_file 0 +workingset_nodereclaim 0 +pgscan 0 +pgsteal 0 +pgscan_kswapd 0 +pgscan_direct 0 +pgscan_khugepaged 0 +pgsteal_kswapd 0 +pgsteal_direct 0 +pgsteal_khugepaged 0 +pgfault 43026352 +pgmajfault 36762 +pgrefill 0 +pgactivate 0 +pgdeactivate 0 +pglazyfree 259 +pglazyfreed 0 +zswpin 0 +zswpout 0 +thp_fault_alloc 0 +thp_collapse_alloc 0 +)"}; + +const std::string EXPECTED[2] + = {"{\"active_anon\": 0, \"active_file\": 1653915648, \"cache\": 4673703936, \"dirty\": 4730880, \"hierarchical_memory_limit\": " + "8589934592, \"hierarchical_memsw_limit\": 8589934592, \"inactive_anon\": 2156335104, \"inactive_file\": 2841305088, " + "\"mapped_file\": 344678400, \"pgfault\": 2055373287, \"pgmajfault\": 0, \"pgpgin\": 2038569918, \"pgpgout\": 2036883790, \"rss\": " + "2232029184, \"rss_huge\": 0, \"shmem\": 0, \"swap\": 0, \"total_active_anon\": 0, \"total_active_file\": 1653915648, " + "\"total_cache\": 4673703936, \"total_dirty\": 4730880, \"total_inactive_anon\": 2156335104, \"total_inactive_file\": 2841305088, " + "\"total_mapped_file\": 344678400, \"total_pgfault\": 2055373287, \"total_pgmajfault\": 0, \"total_pgpgin\": 2038569918, " + "\"total_pgpgout\": 2036883790, \"total_rss\": 2232029184, \"total_rss_huge\": 0, \"total_shmem\": 0, \"total_swap\": 0, " + "\"total_unevictable\": 256008192, \"total_writeback\": 135168, \"unevictable\": 256008192, \"writeback\": 135168}", + "{\"active_anon\": 10429370368, \"active_file\": 8717561856, \"anon\": 10429399040, \"anon_thp\": 0, \"file\": 17410793472, " + "\"file_dirty\": 2060857344, \"file_mapped\": 344010752, \"file_thp\": 0, \"file_writeback\": 0, \"inactive_anon\": 0, " + "\"inactive_file\": 8693084160, \"kernel\": 1537789952, \"kernel_stack\": 3833856, \"pagetables\": 65441792, \"percpu\": 15232, " + "\"pgactivate\": 0, \"pgdeactivate\": 0, \"pgfault\": 43026352, \"pglazyfree\": 259, \"pglazyfreed\": 0, \"pgmajfault\": 36762, " + "\"pgrefill\": 0, \"pgscan\": 0, \"pgscan_direct\": 0, \"pgscan_khugepaged\": 0, \"pgscan_kswapd\": 0, \"pgsteal\": 0, " + "\"pgsteal_direct\": 0, \"pgsteal_khugepaged\": 0, \"pgsteal_kswapd\": 0, \"sec_pagetables\": 0, \"shmem\": 0, \"shmem_thp\": 0, " + "\"slab\": 1466135368, \"slab_reclaimable\": 1460982504, \"slab_unreclaimable\": 5152864, \"sock\": 0, \"swapcached\": 0, " + "\"thp_collapse_alloc\": 0, \"thp_fault_alloc\": 0, \"unevictable\": 0, \"vmalloc\": 0, \"workingset_activate_anon\": 0, " + "\"workingset_activate_file\": 0, \"workingset_nodereclaim\": 0, \"workingset_refault_anon\": 0, \"workingset_refault_file\": 0, " + "\"workingset_restore_anon\": 0, \"workingset_restore_file\": 0, \"zswap\": 0, \"zswapped\": 0, \"zswpin\": 0, \"zswpout\": 0}"}; + + +class CgroupsMemoryUsageObserverFixture : public ::testing::TestWithParam +{ + void SetUp() override + { + const uint8_t version = static_cast(GetParam()); + tmp_dir = fmt::format("./test_cgroups_{}", magic_enum::enum_name(GetParam())); + fs::create_directories(tmp_dir); + + auto stat_file = WriteBufferFromFile(tmp_dir + "/memory.stat"); + stat_file.write(SAMPLE_FILE[version].data(), SAMPLE_FILE[version].size()); + stat_file.sync(); + + if (GetParam() == CgroupsMemoryUsageObserver::CgroupsVersion::V2) + { + auto current_file = WriteBufferFromFile(tmp_dir + "/memory.current"); + current_file.write("29645422592", 11); + current_file.sync(); + } + } + +protected: + std::string tmp_dir; +}; + + +TEST_P(CgroupsMemoryUsageObserverFixture, ReadMemoryUsageTest) +{ + const auto version = GetParam(); + auto reader = createCgroupsReader(version, tmp_dir); + ASSERT_EQ( + reader->readMemoryUsage(), + version == CgroupsMemoryUsageObserver::CgroupsVersion::V1 ? /* rss from memory.stat */ 2232029184 + : /* value from memory.current - inactive_file */ 20952338432); +} + + +TEST_P(CgroupsMemoryUsageObserverFixture, DumpAllStatsTest) +{ + const auto version = GetParam(); + auto reader = createCgroupsReader(version, tmp_dir); + ASSERT_EQ(reader->dumpAllStats(), EXPECTED[static_cast(version)]); +} + + +INSTANTIATE_TEST_SUITE_P( + CgroupsMemoryUsageObserverTests, + CgroupsMemoryUsageObserverFixture, + ::testing::Values(CgroupsMemoryUsageObserver::CgroupsVersion::V1, CgroupsMemoryUsageObserver::CgroupsVersion::V2)); From 3abf636853e755d0a5264eb568ce895e55749920 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 19 Jul 2024 19:16:53 +0200 Subject: [PATCH 31/59] fix for 992 and friends --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../00992_system_parts_race_condition_zookeeper_long.sh | 2 +- tests/queries/0_stateless/replication.lib | 5 ++++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 72f725965e0..416100def4c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3940,7 +3940,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() merge_selecting_task->schedule(); else { - LOG_TRACE(log, "Scheduling next merge selecting task after {}ms", merge_selecting_sleep_ms); + LOG_TRACE(log, "Scheduling next merge selecting task after {}ms, current attempt status: {}", merge_selecting_sleep_ms, result); merge_selecting_task->scheduleAfter(merge_selecting_sleep_ms); } } diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 4887c409844..02a739ece4a 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -41,7 +41,7 @@ function thread3() function thread4() { - while true; do $CLICKHOUSE_CLIENT --receive_timeout=3 -q "OPTIMIZE TABLE alter_table0 FINAL" | grep -Fv "Timeout exceeded while receiving data from server"; done + while true; do $CLICKHOUSE_CLIENT --receive_timeout=1 -q "OPTIMIZE TABLE alter_table0 FINAL" | grep -Fv "Timeout exceeded while receiving data from server"; done } function thread5() diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 2e21f351d2a..6331a5eb406 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -51,6 +51,9 @@ function check_replication_consistency() table_name_prefix=$1 check_query_part=$2 + # Try to kill some mutations because sometimes tests run too much (it's not guarenteed to kill all mutations, see below) + ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null + # Wait for all queries to finish (query may still be running if thread is killed by timeout) num_tries=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%'") -ne 1 ]]; do @@ -94,7 +97,7 @@ function check_replication_consistency() some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1") $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA $some_table PULL" 1>/dev/null 2>/dev/null ||: - # Forcefully cancel mutations to avoid waiting for them to finish + # Forcefully cancel mutations to avoid waiting for them to finish. Kills the remaining mutations ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet From f71244c85c76425bdeeb89ce017caed512f9576c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 08:53:54 +0200 Subject: [PATCH 32/59] Something is strange with the test about refreshable materialized views --- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- .../0_stateless/02932_refreshable_materialized_views.sh | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 857cfd78910..cff0e2cf40b 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -304,7 +304,7 @@ void RefreshTask::refreshTask() { PreformattedMessage message = getCurrentExceptionMessageAndPattern(true); auto text = message.text; - message.text = fmt::format("Refresh failed: {}", message.text); + message.text = fmt::format("Refresh view {} failed: {}", view->getStorageID().getFullTableName(), message.text); LOG_ERROR(log, message); exception = text; } diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 89942e25b67..36cdc8d88e3 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -2,8 +2,6 @@ # Tags: atomic-database CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh @@ -172,13 +170,13 @@ $CLICKHOUSE_CLIENT -nq " drop table b; create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; drop table src;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Exception' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'c' -- $LINENO" | xargs`" != 'Exception' ] do sleep 0.1 done # Check exception, create src, expect successful refresh. $CLICKHOUSE_CLIENT -nq " - select '<19: exception>', exception ilike '%UNKNOWN_TABLE%' from refreshes; + select '<19: exception>', exception ilike '%UNKNOWN_TABLE%' ? '1' : exception from refreshes where view = 'c'; create table src (x Int64) engine Memory as select 1; system refresh view c;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] From fab0bb6aa41c9deccb69666ae0d22559c0784121 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 09:01:44 +0200 Subject: [PATCH 33/59] Remove all sleeps --- .../02932_refreshable_materialized_views.sh | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 36cdc8d88e3..9081035579d 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -132,7 +132,7 @@ while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshe do sleep 0.1 done -sleep 1 + $CLICKHOUSE_CLIENT -nq " select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; truncate src; @@ -222,22 +222,27 @@ done $CLICKHOUSE_CLIENT -nq " rename table e to f; select '<24: rename during refresh>', * from f; - select '<25: rename during refresh>', view, status from refreshes; + select '<25: rename during refresh>', view, status from refreshes where view = 'f'; alter table f modify refresh after 10 year;" -sleep 2 # make it likely that at least one row was processed + # Cancel. $CLICKHOUSE_CLIENT -nq " system cancel view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Cancelled' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Cancelled' ] do sleep 0.1 done + +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" = 'Running' ] +do + sleep 0.1 +done + # Check that another refresh doesn't immediately start after the cancelled one. -sleep 1 $CLICKHOUSE_CLIENT -nq " - select '<27: cancelled>', view, status from refreshes; + select '<27: cancelled>', view, status from refreshes where view = 'f'; system refresh view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Running' ] do sleep 0.1 done From 7643b99dc5a892f404d756e92c4a17cf689c720b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 06:08:00 +0200 Subject: [PATCH 34/59] Split a test for index --- .../0_stateless/02995_index_1.reference | 15 +++++++ tests/queries/0_stateless/02995_index_1.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_10.reference | 13 ++++++ tests/queries/0_stateless/02995_index_10.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_2.reference | 13 ++++++ tests/queries/0_stateless/02995_index_2.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_3.reference | 16 +++++++ tests/queries/0_stateless/02995_index_3.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_4.reference | 16 +++++++ tests/queries/0_stateless/02995_index_4.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_5.reference | 9 ++++ tests/queries/0_stateless/02995_index_5.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_6.reference | 14 ++++++ tests/queries/0_stateless/02995_index_6.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_7.reference | 9 ++++ tests/queries/0_stateless/02995_index_7.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_8.reference | 12 +++++ tests/queries/0_stateless/02995_index_8.sh | 44 +++++++++++++++++++ .../0_stateless/02995_index_9.reference | 9 ++++ tests/queries/0_stateless/02995_index_9.sh | 44 +++++++++++++++++++ 20 files changed, 566 insertions(+) create mode 100644 tests/queries/0_stateless/02995_index_1.reference create mode 100755 tests/queries/0_stateless/02995_index_1.sh create mode 100644 tests/queries/0_stateless/02995_index_10.reference create mode 100755 tests/queries/0_stateless/02995_index_10.sh create mode 100644 tests/queries/0_stateless/02995_index_2.reference create mode 100755 tests/queries/0_stateless/02995_index_2.sh create mode 100644 tests/queries/0_stateless/02995_index_3.reference create mode 100755 tests/queries/0_stateless/02995_index_3.sh create mode 100644 tests/queries/0_stateless/02995_index_4.reference create mode 100755 tests/queries/0_stateless/02995_index_4.sh create mode 100644 tests/queries/0_stateless/02995_index_5.reference create mode 100755 tests/queries/0_stateless/02995_index_5.sh create mode 100644 tests/queries/0_stateless/02995_index_6.reference create mode 100755 tests/queries/0_stateless/02995_index_6.sh create mode 100644 tests/queries/0_stateless/02995_index_7.reference create mode 100755 tests/queries/0_stateless/02995_index_7.sh create mode 100644 tests/queries/0_stateless/02995_index_8.reference create mode 100755 tests/queries/0_stateless/02995_index_8.sh create mode 100644 tests/queries/0_stateless/02995_index_9.reference create mode 100755 tests/queries/0_stateless/02995_index_9.sh diff --git a/tests/queries/0_stateless/02995_index_1.reference b/tests/queries/0_stateless/02995_index_1.reference new file mode 100644 index 00000000000..6c3b1230db6 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_1.reference @@ -0,0 +1,15 @@ +12 4 21722 2209341 4 1415 2333 4 61 64 3 +21 1134 11363 58484 1106 1458 1592 136 26 62 32 +22 210 4504 5729 196 291 767 124 47 54 8 +26 196 1327684 5221 195 4140 5661 161 28 49 19 +28 5 2034378 7102 5 325 3255 2 53 60 4 +29 53 45041 45189 45 1580 211 31 55 84 18 +38 424 1600675 4653 424 562 5944 244 60 65 6 +45 17 62743 674873 17 6239 6494 17 65 76 8 +72 1862 1210073 6200 1677 2498 528 859 51 61 11 +79 2 2255228 2255293 2 5495 7057 2 65 65 1 +85 459 1051571 1829638 459 6402 7131 334 32 61 25 +86 10 1748130 1754217 10 4374 7003 10 56 59 4 +91 165 5718 5802 75 282 7113 112 41 63 22 +94 20 1231916 2050003 20 4802 4917 19 53 59 7 +99 2 3665 36667 2 497 697 2 70 71 2 diff --git a/tests/queries/0_stateless/02995_index_1.sh b/tests/queries/0_stateless/02995_index_1.sh new file mode 100755 index 00000000000..a5f1b30c2e8 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_1.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {1..100} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_10.reference b/tests/queries/0_stateless/02995_index_10.reference new file mode 100644 index 00000000000..bfa38d03801 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_10.reference @@ -0,0 +1,13 @@ +912 146 1322641 2238040 146 1366 6354 143 59 59 1 +913 82 5495 6870 78 350 565 67 24 43 15 +921 763 1580790 416881 763 6191 7131 509 63 64 2 +925 318 2500952 5025 309 476 6114 182 32 56 21 +931 12 4277 4809 12 238 256 9 63 83 9 +942 954 1331 2228193 952 1121 5047 788 65 70 6 +948 14 1785593 2600431 14 6550 6598 13 34 49 9 +956 5 5755 6023 5 359 411 5 43 48 4 +963 4 3812 3835 4 444 537 4 47 53 4 +978 5 51632 58212 5 1127 1556 5 24 32 5 +980 53 47201 59744 53 1537 1625 36 41 49 9 +987 6033 2020131 763444 4306 256 792 1832 60 64 5 +993 4 1615159 1718339 4 1570 3093 4 62 63 2 diff --git a/tests/queries/0_stateless/02995_index_10.sh b/tests/queries/0_stateless/02995_index_10.sh new file mode 100755 index 00000000000..d72c7c72705 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_10.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {901..1000} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_2.reference b/tests/queries/0_stateless/02995_index_2.reference new file mode 100644 index 00000000000..098292a289f --- /dev/null +++ b/tests/queries/0_stateless/02995_index_2.reference @@ -0,0 +1,13 @@ +103 1 2446615 2446615 1 2498 2498 1 58 58 1 +106 72 6149 6699 67 527 826 40 61 61 1 +111 43 2273186 5272 43 492 4923 4 54 72 15 +120 3129 45117 6735 2868 1030 1625 561 59 64 6 +138 2 49243 49374 2 1428 1519 2 47 48 2 +143 100 23321 63639 100 1115 1624 88 51 51 1 +145 1 2447976 2447976 1 6173 6173 1 44 44 1 +153 16 13748 16881 16 1506 1636 16 54 68 9 +159 19952 1525336 7131 12957 1280 6163 2668 24 66 39 +171 5 15042 16698 5 1302 1608 5 65 65 1 +179 6264 1362341 2686 6244 2554 7132 2705 61 67 7 +192 1 1639623 1639623 1 3406 3406 1 32 32 1 +193 1 1429969 1429969 1 7131 7131 1 45 45 1 diff --git a/tests/queries/0_stateless/02995_index_2.sh b/tests/queries/0_stateless/02995_index_2.sh new file mode 100755 index 00000000000..e7451c7ee4b --- /dev/null +++ b/tests/queries/0_stateless/02995_index_2.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {101..200} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_3.reference b/tests/queries/0_stateless/02995_index_3.reference new file mode 100644 index 00000000000..9c2fca9fde6 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_3.reference @@ -0,0 +1,16 @@ +207 12 23057 32500 12 1491 1726 12 32 46 7 +221 5081 1366870 6649 3432 4527 5226 687 24 69 39 +228 73 12281 17929 71 1328 2034 63 49 71 18 +229 2 1617478 1723791 2 4590 5578 2 41 42 2 +230 3916 1332729 6949 3668 1330 4703 845 62 65 4 +238 25 2624456 2625673 24 2535 6465 25 58 75 14 +241 154 2554929 2616444 154 2626 7131 148 34 57 17 +248 276 15529 30987 274 1040 1222 136 37 79 27 +254 3018 33966 6635 2837 1057 1622 539 24 60 33 +255 20 1581774 1811334 20 6068 6301 18 33 57 10 +256 5 5145 6841 5 367 376 5 58 58 1 +270 2 2195579 2262119 2 7102 7123 2 33 34 2 +281 32 2379460 616215 32 6042 6086 23 53 64 12 +282 7 1292651 24244 7 1607 2455 6 46 55 5 +286 123 1521935 5269 123 3793 3940 81 40 66 22 +291 21 2419080 3567 21 297 4731 21 54 55 2 diff --git a/tests/queries/0_stateless/02995_index_3.sh b/tests/queries/0_stateless/02995_index_3.sh new file mode 100755 index 00000000000..506429e2696 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_3.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {201..300} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_4.reference b/tests/queries/0_stateless/02995_index_4.reference new file mode 100644 index 00000000000..deff7afaed3 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_4.reference @@ -0,0 +1,16 @@ +316 4 5221 5616 4 505 558 4 32 35 3 +319 232 56480 63033 230 1599 313 50 33 64 26 +327 15 51647 51894 14 1292 1585 14 47 57 7 +332 24 23484 54948 24 1609 1726 16 32 49 11 +333 1 14189 14189 1 1550 1550 1 63 63 1 +342 49 2579220 2622432 49 4626 6933 48 34 54 14 +344 1 6486 6486 1 509 509 1 24 24 1 +346 1987 53016 6735 1823 1334 174 294 26 62 32 +358 45 59058 60844 44 6746 722 40 57 84 15 +363 1198 1260033 2568811 1196 5710 5790 82 55 80 26 +384 150 2361175 476024 150 7008 7123 81 38 64 22 +387 277 5200 6553 252 243 521 130 65 65 1 +392 1877 1607428 2030850 1875 1416 7131 1379 54 66 13 +396 8181 1380803 6186 7920 545 798 1743 24 67 39 +398 3 5183 5213 2 291 352 3 53 59 3 +399 62 51494 59203 61 7073 754 42 55 78 18 diff --git a/tests/queries/0_stateless/02995_index_4.sh b/tests/queries/0_stateless/02995_index_4.sh new file mode 100755 index 00000000000..1a0458728f9 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_4.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {301..400} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_5.reference b/tests/queries/0_stateless/02995_index_5.reference new file mode 100644 index 00000000000..c5ab4d2417c --- /dev/null +++ b/tests/queries/0_stateless/02995_index_5.reference @@ -0,0 +1,9 @@ +412 2141 1360120 2189792 2136 2491 5658 1371 71 75 5 +413 2 2036037 2064917 2 3963 4666 2 43 45 2 +431 33 2302331 2348449 33 4425 6516 32 69 69 1 +447 59 25125 33094 59 1176 1817 56 53 58 6 +456 1 53157 53157 1 1556 1556 1 26 26 1 +462 5 5456 6280 5 348 4337 5 28 40 5 +472 1 1443716 1443716 1 6122 6122 1 42 42 1 +491 34 1066102 1183673 34 6606 6822 32 46 67 15 +498 896 2230163 3054 895 537 7131 714 24 59 28 diff --git a/tests/queries/0_stateless/02995_index_5.sh b/tests/queries/0_stateless/02995_index_5.sh new file mode 100755 index 00000000000..60c12a8146d --- /dev/null +++ b/tests/queries/0_stateless/02995_index_5.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {401..500} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_6.reference b/tests/queries/0_stateless/02995_index_6.reference new file mode 100644 index 00000000000..bac19179bb6 --- /dev/null +++ b/tests/queries/0_stateless/02995_index_6.reference @@ -0,0 +1,14 @@ +504 108 12281 25180 108 1318 1784 94 55 66 12 +515 22 1588883 2640809 22 6554 6571 15 46 59 12 +518 1 37743 37743 1 1558 1558 1 72 72 1 +530 1 3033 3033 1 561 561 1 59 59 1 +532 26 5721 6355 25 549 665 14 44 50 7 +546 156 2577874 48517 156 1105 324 133 44 51 8 +554 12 1665194 2640066 12 1817 2951 12 57 57 1 +564 3865 2028049 2083433 3722 1115 985 2203 44 84 41 +566 4432 50605 57509 3217 1191 267 459 26 72 39 +567 8 5221 5893 7 333 558 8 27 35 4 +582 1172 1320619 2019743 1172 5819 7131 757 26 63 30 +584 43100 2500 5594 22561 134 4573 1660 48 84 37 +589 28 6046 6068 19 345 564 27 55 62 8 +595 139 1585165 1683606 138 2231 3598 132 54 84 28 diff --git a/tests/queries/0_stateless/02995_index_6.sh b/tests/queries/0_stateless/02995_index_6.sh new file mode 100755 index 00000000000..4936f73f36b --- /dev/null +++ b/tests/queries/0_stateless/02995_index_6.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {501..600} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_7.reference b/tests/queries/0_stateless/02995_index_7.reference new file mode 100644 index 00000000000..4f1d28ab37e --- /dev/null +++ b/tests/queries/0_stateless/02995_index_7.reference @@ -0,0 +1,9 @@ +615 3 1056081 1116230 3 5794 5796 2 59 62 3 +619 7 1543114 5241 7 2442 3105 7 41 45 3 +634 2722 1221058 4999 2686 2426 7131 1735 54 60 7 +635 237 2119333 4667 237 561 5999 176 49 60 12 +644 5 1774169 2056171 5 5591 6091 4 33 39 3 +647 8 51632 64403 8 1457 1624 8 26 34 5 +651 1325 1620565 6281 1301 528 792 815 62 63 2 +665 13 4598 4789 13 511 558 11 39 46 7 +679 1560 1613200 25940 1552 1569 3118 781 49 84 35 diff --git a/tests/queries/0_stateless/02995_index_7.sh b/tests/queries/0_stateless/02995_index_7.sh new file mode 100755 index 00000000000..26be310abce --- /dev/null +++ b/tests/queries/0_stateless/02995_index_7.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {601..700} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_8.reference b/tests/queries/0_stateless/02995_index_8.reference new file mode 100644 index 00000000000..62fbfb2af9a --- /dev/null +++ b/tests/queries/0_stateless/02995_index_8.reference @@ -0,0 +1,12 @@ +704 2 14226 15594 2 1086 1116 2 65 71 2 +715 25 1199352 3490 25 5036 5112 23 34 55 13 +716 1253 61989 6735 1050 1203 1625 397 52 65 14 +730 2584 5560 6170 634 2421 627 293 56 69 14 +736 8 1433153 4941 8 339 4594 8 28 36 5 +749 2 1326176 1339862 2 4339 6213 2 49 50 2 +753 1 53157 53157 1 1556 1556 1 26 26 1 +761 63 1443230 6881 63 3154 3204 26 56 73 14 +762 49 1449596 1968154 49 2437 3753 48 54 62 9 +775 35107 5330 769436 2471 447 6607 656 70 81 12 +789 1 1552458 1552458 1 2441 2441 1 62 62 1 +794 158 5585 6585 155 495 929 67 24 50 20 diff --git a/tests/queries/0_stateless/02995_index_8.sh b/tests/queries/0_stateless/02995_index_8.sh new file mode 100755 index 00000000000..8c2620b59fd --- /dev/null +++ b/tests/queries/0_stateless/02995_index_8.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {701..800} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_9.reference b/tests/queries/0_stateless/02995_index_9.reference new file mode 100644 index 00000000000..86c7be173bd --- /dev/null +++ b/tests/queries/0_stateless/02995_index_9.reference @@ -0,0 +1,9 @@ +839 9 29223 46530 9 1336 1465 9 52 52 1 +844 5 2377545 2377635 5 5129 6321 5 53 69 5 +846 50 2172273 2589295 50 1582 3053 48 64 68 5 +847 2577 56656 63658 1582 1444 838 474 26 63 33 +861 1333 5570 6909 839 457 489 37 33 70 34 +873 2360 1519811 50487 2248 1310 1784 316 60 68 9 +879 228 6704 6785 79 279 507 121 35 66 24 +889 5130 2070007 39692 5040 1151 6791 2606 44 66 23 +896 4 511246 859452 4 6554 6561 4 67 71 4 diff --git a/tests/queries/0_stateless/02995_index_9.sh b/tests/queries/0_stateless/02995_index_9.sh new file mode 100755 index 00000000000..76160c62aaa --- /dev/null +++ b/tests/queries/0_stateless/02995_index_9.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +INSERT INTO test +SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) +FROM numbers(100000); + +DETACH TABLE test; +ATTACH TABLE test; +" + +for i in {801..900} +do + echo " +WITH ${i} AS try +SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test +WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String + AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String + AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String + AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String + AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String + AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String +HAVING count() > 0; +" +done | ${CLICKHOUSE_CLIENT} --multiquery + +${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" From 70d6320577a483d6011872a5f642927c27d095c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 06:08:16 +0200 Subject: [PATCH 35/59] Split a test for index --- .../queries/0_stateless/02995_index.reference | 126 ------------------ tests/queries/0_stateless/02995_index.sh | 44 ------ 2 files changed, 170 deletions(-) delete mode 100644 tests/queries/0_stateless/02995_index.reference delete mode 100755 tests/queries/0_stateless/02995_index.sh diff --git a/tests/queries/0_stateless/02995_index.reference b/tests/queries/0_stateless/02995_index.reference deleted file mode 100644 index 1e8639caa88..00000000000 --- a/tests/queries/0_stateless/02995_index.reference +++ /dev/null @@ -1,126 +0,0 @@ -12 4 21722 2209341 4 1415 2333 4 61 64 3 -21 1134 11363 58484 1106 1458 1592 136 26 62 32 -22 210 4504 5729 196 291 767 124 47 54 8 -26 196 1327684 5221 195 4140 5661 161 28 49 19 -28 5 2034378 7102 5 325 3255 2 53 60 4 -29 53 45041 45189 45 1580 211 31 55 84 18 -38 424 1600675 4653 424 562 5944 244 60 65 6 -45 17 62743 674873 17 6239 6494 17 65 76 8 -72 1862 1210073 6200 1677 2498 528 859 51 61 11 -79 2 2255228 2255293 2 5495 7057 2 65 65 1 -85 459 1051571 1829638 459 6402 7131 334 32 61 25 -86 10 1748130 1754217 10 4374 7003 10 56 59 4 -91 165 5718 5802 75 282 7113 112 41 63 22 -94 20 1231916 2050003 20 4802 4917 19 53 59 7 -99 2 3665 36667 2 497 697 2 70 71 2 -103 1 2446615 2446615 1 2498 2498 1 58 58 1 -106 72 6149 6699 67 527 826 40 61 61 1 -111 43 2273186 5272 43 492 4923 4 54 72 15 -120 3129 45117 6735 2868 1030 1625 561 59 64 6 -138 2 49243 49374 2 1428 1519 2 47 48 2 -143 100 23321 63639 100 1115 1624 88 51 51 1 -145 1 2447976 2447976 1 6173 6173 1 44 44 1 -153 16 13748 16881 16 1506 1636 16 54 68 9 -159 19952 1525336 7131 12957 1280 6163 2668 24 66 39 -171 5 15042 16698 5 1302 1608 5 65 65 1 -179 6264 1362341 2686 6244 2554 7132 2705 61 67 7 -192 1 1639623 1639623 1 3406 3406 1 32 32 1 -193 1 1429969 1429969 1 7131 7131 1 45 45 1 -207 12 23057 32500 12 1491 1726 12 32 46 7 -221 5081 1366870 6649 3432 4527 5226 687 24 69 39 -228 73 12281 17929 71 1328 2034 63 49 71 18 -229 2 1617478 1723791 2 4590 5578 2 41 42 2 -230 3916 1332729 6949 3668 1330 4703 845 62 65 4 -238 25 2624456 2625673 24 2535 6465 25 58 75 14 -241 154 2554929 2616444 154 2626 7131 148 34 57 17 -248 276 15529 30987 274 1040 1222 136 37 79 27 -254 3018 33966 6635 2837 1057 1622 539 24 60 33 -255 20 1581774 1811334 20 6068 6301 18 33 57 10 -256 5 5145 6841 5 367 376 5 58 58 1 -270 2 2195579 2262119 2 7102 7123 2 33 34 2 -281 32 2379460 616215 32 6042 6086 23 53 64 12 -282 7 1292651 24244 7 1607 2455 6 46 55 5 -286 123 1521935 5269 123 3793 3940 81 40 66 22 -291 21 2419080 3567 21 297 4731 21 54 55 2 -316 4 5221 5616 4 505 558 4 32 35 3 -319 232 56480 63033 230 1599 313 50 33 64 26 -327 15 51647 51894 14 1292 1585 14 47 57 7 -332 24 23484 54948 24 1609 1726 16 32 49 11 -333 1 14189 14189 1 1550 1550 1 63 63 1 -342 49 2579220 2622432 49 4626 6933 48 34 54 14 -344 1 6486 6486 1 509 509 1 24 24 1 -346 1987 53016 6735 1823 1334 174 294 26 62 32 -358 45 59058 60844 44 6746 722 40 57 84 15 -363 1198 1260033 2568811 1196 5710 5790 82 55 80 26 -384 150 2361175 476024 150 7008 7123 81 38 64 22 -387 277 5200 6553 252 243 521 130 65 65 1 -392 1877 1607428 2030850 1875 1416 7131 1379 54 66 13 -396 8181 1380803 6186 7920 545 798 1743 24 67 39 -398 3 5183 5213 2 291 352 3 53 59 3 -399 62 51494 59203 61 7073 754 42 55 78 18 -412 2141 1360120 2189792 2136 2491 5658 1371 71 75 5 -413 2 2036037 2064917 2 3963 4666 2 43 45 2 -431 33 2302331 2348449 33 4425 6516 32 69 69 1 -447 59 25125 33094 59 1176 1817 56 53 58 6 -456 1 53157 53157 1 1556 1556 1 26 26 1 -462 5 5456 6280 5 348 4337 5 28 40 5 -472 1 1443716 1443716 1 6122 6122 1 42 42 1 -491 34 1066102 1183673 34 6606 6822 32 46 67 15 -498 896 2230163 3054 895 537 7131 714 24 59 28 -504 108 12281 25180 108 1318 1784 94 55 66 12 -515 22 1588883 2640809 22 6554 6571 15 46 59 12 -518 1 37743 37743 1 1558 1558 1 72 72 1 -530 1 3033 3033 1 561 561 1 59 59 1 -532 26 5721 6355 25 549 665 14 44 50 7 -546 156 2577874 48517 156 1105 324 133 44 51 8 -554 12 1665194 2640066 12 1817 2951 12 57 57 1 -564 3865 2028049 2083433 3722 1115 985 2203 44 84 41 -566 4432 50605 57509 3217 1191 267 459 26 72 39 -567 8 5221 5893 7 333 558 8 27 35 4 -582 1172 1320619 2019743 1172 5819 7131 757 26 63 30 -584 43100 2500 5594 22561 134 4573 1660 48 84 37 -589 28 6046 6068 19 345 564 27 55 62 8 -595 139 1585165 1683606 138 2231 3598 132 54 84 28 -615 3 1056081 1116230 3 5794 5796 2 59 62 3 -619 7 1543114 5241 7 2442 3105 7 41 45 3 -634 2722 1221058 4999 2686 2426 7131 1735 54 60 7 -635 237 2119333 4667 237 561 5999 176 49 60 12 -644 5 1774169 2056171 5 5591 6091 4 33 39 3 -647 8 51632 64403 8 1457 1624 8 26 34 5 -651 1325 1620565 6281 1301 528 792 815 62 63 2 -665 13 4598 4789 13 511 558 11 39 46 7 -679 1560 1613200 25940 1552 1569 3118 781 49 84 35 -704 2 14226 15594 2 1086 1116 2 65 71 2 -715 25 1199352 3490 25 5036 5112 23 34 55 13 -716 1253 61989 6735 1050 1203 1625 397 52 65 14 -730 2584 5560 6170 634 2421 627 293 56 69 14 -736 8 1433153 4941 8 339 4594 8 28 36 5 -749 2 1326176 1339862 2 4339 6213 2 49 50 2 -753 1 53157 53157 1 1556 1556 1 26 26 1 -761 63 1443230 6881 63 3154 3204 26 56 73 14 -762 49 1449596 1968154 49 2437 3753 48 54 62 9 -775 35107 5330 769436 2471 447 6607 656 70 81 12 -789 1 1552458 1552458 1 2441 2441 1 62 62 1 -794 158 5585 6585 155 495 929 67 24 50 20 -839 9 29223 46530 9 1336 1465 9 52 52 1 -844 5 2377545 2377635 5 5129 6321 5 53 69 5 -846 50 2172273 2589295 50 1582 3053 48 64 68 5 -847 2577 56656 63658 1582 1444 838 474 26 63 33 -861 1333 5570 6909 839 457 489 37 33 70 34 -873 2360 1519811 50487 2248 1310 1784 316 60 68 9 -879 228 6704 6785 79 279 507 121 35 66 24 -889 5130 2070007 39692 5040 1151 6791 2606 44 66 23 -896 4 511246 859452 4 6554 6561 4 67 71 4 -912 146 1322641 2238040 146 1366 6354 143 59 59 1 -913 82 5495 6870 78 350 565 67 24 43 15 -921 763 1580790 416881 763 6191 7131 509 63 64 2 -925 318 2500952 5025 309 476 6114 182 32 56 21 -931 12 4277 4809 12 238 256 9 63 83 9 -942 954 1331 2228193 952 1121 5047 788 65 70 6 -948 14 1785593 2600431 14 6550 6598 13 34 49 9 -956 5 5755 6023 5 359 411 5 43 48 4 -963 4 3812 3835 4 444 537 4 47 53 4 -978 5 51632 58212 5 1127 1556 5 24 32 5 -980 53 47201 59744 53 1537 1625 36 41 49 9 -987 6033 2020131 763444 4306 256 792 1832 60 64 5 -993 4 1615159 1718339 4 1570 3093 4 62 63 2 diff --git a/tests/queries/0_stateless/02995_index.sh b/tests/queries/0_stateless/02995_index.sh deleted file mode 100755 index 5125d03904e..00000000000 --- a/tests/queries/0_stateless/02995_index.sh +++ /dev/null @@ -1,44 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} --multiquery " - -DROP TABLE IF EXISTS test; -CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; - -INSERT INTO test -SELECT round(pow(sipHash64(1, number), 1/4)), round(pow(sipHash64(2, number), 1/6)), round(pow(sipHash64(3, number), 1/10)) -FROM numbers(100000); - -INSERT INTO test -SELECT round(pow(sipHash64(1, number), 1/3)), round(pow(sipHash64(2, number), 1/5)), round(pow(sipHash64(3, number), 1/10)) -FROM numbers(100000); - -INSERT INTO test -SELECT round(pow(sipHash64(1, number), 1/5)), round(pow(sipHash64(2, number), 1/7)), round(pow(sipHash64(3, number), 1/10)) -FROM numbers(100000); - -DETACH TABLE test; -ATTACH TABLE test; -" - -for i in {1..1000} -do - echo " -WITH ${i} AS try -SELECT try, count(), min(a), max(a), uniqExact(a), min(b), max(b), uniqExact(b), min(c), max(c), uniqExact(c) FROM test -WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1)::String - AND a <= (a1 + round(pow(sipHash64(3, try), 1 / (3 + sipHash64(4, try) % 8))))::String - AND b >= (round(pow(sipHash64(5, try), 1 / (3 + sipHash64(6, try) % 8))) AS b1)::String - AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String - AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String - AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String -HAVING count() > 0; -" -done | ${CLICKHOUSE_CLIENT} --multiquery - -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" From 1b606e525a9942ef69b0210760380170291d4f1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 06:34:10 +0200 Subject: [PATCH 36/59] Fix inconsistent formatting of `NOT ((SELECT ...))` --- src/Parsers/ASTFunction.cpp | 12 ++++++++---- ...inconsistent_formatting_of_not_subquery.reference | 1 + .../03208_inconsistent_formatting_of_not_subquery.sh | 9 +++++++++ 3 files changed, 18 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.reference create mode 100755 tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index b04ec1c22b2..230d4c778e8 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -329,19 +329,23 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format const auto * literal = arguments->children[0]->as(); const auto * function = arguments->children[0]->as(); + const auto * subquery = arguments->children[0]->as(); bool is_tuple = literal && literal->value.getType() == Field::Types::Tuple; - // do not add parentheses for tuple literal, otherwise extra parens will be added `-((3, 7, 3), 1)` -> `-(((3, 7, 3), 1))` + /// Do not add parentheses for tuple literal, otherwise extra parens will be added `-((3, 7, 3), 1)` -> `-(((3, 7, 3), 1))` bool literal_need_parens = literal && !is_tuple; - // negate always requires parentheses, otherwise -(-1) will be printed as --1 - bool inside_parens = name == "negate" && (literal_need_parens || (function && function->name == "negate")); + /// Negate always requires parentheses, otherwise -(-1) will be printed as --1 + /// Also extra parentheses are needed for subqueries, because NOT can be parsed as a function: + /// not(SELECT 1) cannot be parsed, while not((SELECT 1)) can. + bool inside_parens = (name == "negate" && (literal_need_parens || (function && function->name == "negate"))) + || (subquery && name == "not"); /// We DO need parentheses around a single literal /// For example, SELECT (NOT 0) + (NOT 0) cannot be transformed into SELECT NOT 0 + NOT 0, since /// this is equal to SELECT NOT (0 + NOT 0) bool outside_parens = frame.need_parens && !inside_parens; - // do not add extra parentheses for functions inside negate, i.e. -(-toUInt64(-(1))) + /// Do not add extra parentheses for functions inside negate, i.e. -(-toUInt64(-(1))) if (inside_parens) nested_need_parens.need_parens = false; diff --git a/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.reference b/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.reference new file mode 100644 index 00000000000..a1afeb1ab82 --- /dev/null +++ b/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.reference @@ -0,0 +1 @@ +SELECT NOT ((SELECT 1)) diff --git a/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh b/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh new file mode 100755 index 00000000000..5f7397015fa --- /dev/null +++ b/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so the grpc library is not built + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_FORMAT --oneline --query "SELECT NOT((SELECT 1))" From efb6491cfdc10c2b8ba6268c29677ea298046bed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 06:34:57 +0200 Subject: [PATCH 37/59] Update 03208_inconsistent_formatting_of_not_subquery.sh --- .../03208_inconsistent_formatting_of_not_subquery.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh b/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh index 5f7397015fa..594d316b621 100755 --- a/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh +++ b/tests/queries/0_stateless/03208_inconsistent_formatting_of_not_subquery.sh @@ -1,6 +1,4 @@ #!/usr/bin/env bash -# Tags: no-fasttest -# Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so the grpc library is not built CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 1be54641c16569e7ee3398fe0770a9b532342643 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 11:03:19 +0200 Subject: [PATCH 38/59] Revert libunwind patch --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 8f28e64d158..fe854449e24 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2 +Subproject commit fe854449e24bedfa26e38465b84374312dbd587f From c36dde51031e817b20db6773df47a42092f6616b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 12:17:29 +0200 Subject: [PATCH 39/59] Fix KeeperMap create after incomplete drop --- src/Common/FailPoint.cpp | 3 ++- src/Storages/StorageKeeperMap.cpp | 27 +++++++++++-------- ...03208_keepermap_failed_data_drop.reference | 0 .../03208_keepermap_failed_data_drop.sql | 7 +++++ 4 files changed, 25 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/03208_keepermap_failed_data_drop.reference create mode 100644 tests/queries/0_stateless/03208_keepermap_failed_data_drop.sql diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 5454cba8e2e..b952e9725e3 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -57,7 +57,8 @@ static struct InitFiu PAUSEABLE_ONCE(finish_clean_quorum_failed_parts) \ PAUSEABLE(dummy_pausable_failpoint) \ ONCE(execute_query_calling_empty_set_result_func_on_exception) \ - ONCE(receive_timeout_on_table_status_response) + ONCE(receive_timeout_on_table_status_response) \ + ONCE(keepermap_fail_drop_data) \ namespace FailPoints diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 587cb621362..1c1de245d10 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -37,6 +37,7 @@ #include #include +#include #include #include #include @@ -64,6 +65,11 @@ namespace DB { +namespace FailPoints +{ + extern const char keepermap_fail_drop_data[]; +} + namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -411,18 +417,13 @@ StorageKeeperMap::StorageKeeperMap( auto code = client->tryCreate(zk_table_path, "", zkutil::CreateMode::Persistent); - // tables_path was removed with drop - if (code == Coordination::Error::ZNONODE) - { - LOG_INFO(log, "Metadata nodes were removed by another server, will retry"); - continue; - } - else if (code != Coordination::Error::ZOK) - { - throw zkutil::KeeperException(code, "Failed to create table on path {} because a table with same UUID already exists", zk_root_path); - } + if (code == Coordination::Error::ZOK) + return; - return; + if (code != Coordination::Error::ZNONODE) + throw zkutil::KeeperException(code, "Failed to create table on path {} because a table with same UUID already exists", zk_root_path); + + /// ZNONODE means we dropped zk_tables_path but didn't finish drop completely } if (client->exists(zk_dropped_path)) @@ -561,6 +562,10 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) { + fiu_do_on(FailPoints::keepermap_fail_drop_data, + { + throw zkutil::KeeperException(Coordination::Error::ZOPERATIONTIMEOUT, "Manually triggered operation timeout"); + }); zookeeper->removeChildrenRecursive(zk_data_path); bool completely_removed = false; diff --git a/tests/queries/0_stateless/03208_keepermap_failed_data_drop.reference b/tests/queries/0_stateless/03208_keepermap_failed_data_drop.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03208_keepermap_failed_data_drop.sql b/tests/queries/0_stateless/03208_keepermap_failed_data_drop.sql new file mode 100644 index 00000000000..ad0603f12a9 --- /dev/null +++ b/tests/queries/0_stateless/03208_keepermap_failed_data_drop.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS 03208_keepermap_test SYNC; + +CREATE TABLE 03208_keepermap_test (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test03208') PRIMARY KEY(key); +INSERT INTO 03208_keepermap_test VALUES (1, 11); +SYSTEM ENABLE FAILPOINT keepermap_fail_drop_data; +DROP TABLE 03208_keepermap_test; +CREATE TABLE 03208_keepermap_test_another (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test03208') PRIMARY KEY(key); From 1ea05e55d25e144ff2294265a4d70c420174004a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 10:19:54 +0000 Subject: [PATCH 40/59] Split test 03038_nested_dynamic_merges to avoid timeouts --- .../03038_nested_dynamic_merges.reference | 92 ------------------- .../03038_nested_dynamic_merges.sh | 53 ----------- ...ynamic_merges_compact_horizontal.reference | 21 +++++ ...ested_dynamic_merges_compact_horizontal.sh | 32 +++++++ ..._dynamic_merges_compact_vertical.reference | 21 +++++ ..._nested_dynamic_merges_compact_vertical.sh | 32 +++++++ ...d_dynamic_merges_wide_horizontal.reference | 21 +++++ ...8_nested_dynamic_merges_wide_horizontal.sh | 32 +++++++ ...ted_dynamic_merges_wide_vertical.reference | 21 +++++ ...038_nested_dynamic_merges_wide_vertical.sh | 32 +++++++ 10 files changed, 212 insertions(+), 145 deletions(-) delete mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges.reference delete mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference create mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference create mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference create mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference create mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges.reference deleted file mode 100644 index 65034647775..00000000000 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges.reference +++ /dev/null @@ -1,92 +0,0 @@ -MergeTree compact + horizontal merge -test -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None -MergeTree wide + horizontal merge -test -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None -MergeTree compact + vertical merge -test -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None -MergeTree wide + vertical merge -test -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges.sh deleted file mode 100755 index b82ddb3813e..00000000000 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - - -function test() -{ - echo "test" - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, number from numbers(100000)" - $CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" - $CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" - - $CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - - $CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" - $CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" - - $CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -} - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree compact + horizontal merge" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide + horizontal merge" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree compact + vertical merge" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide + vertical merge" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" -test -$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference new file mode 100644 index 00000000000..27ed336a035 --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference @@ -0,0 +1,21 @@ +16667 Tuple(a Dynamic(max_types=3)):Date +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):String +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 UInt64:None +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 UInt64:None +16667 Tuple(a Dynamic(max_types=3)):DateTime +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +133333 Tuple(a Dynamic(max_types=3)):None +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +116667 Tuple(a Dynamic(max_types=3)):String +133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh new file mode 100755 index 00000000000..cb8a8fad6ba --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" + +$CH_CLIENT -q "drop table if exists test;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" + +$CH_CLIENT -q "system stop merges test" +$CH_CLIENT -q "insert into test select number, number from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "drop table test;" + diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference new file mode 100644 index 00000000000..27ed336a035 --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference @@ -0,0 +1,21 @@ +16667 Tuple(a Dynamic(max_types=3)):Date +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):String +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 UInt64:None +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 UInt64:None +16667 Tuple(a Dynamic(max_types=3)):DateTime +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +133333 Tuple(a Dynamic(max_types=3)):None +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +116667 Tuple(a Dynamic(max_types=3)):String +133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh new file mode 100755 index 00000000000..0627244a02d --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" + +$CH_CLIENT -q "drop table if exists test;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" + +$CH_CLIENT -q "system stop merges test" +$CH_CLIENT -q "insert into test select number, number from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "drop table test;" + diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference new file mode 100644 index 00000000000..27ed336a035 --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference @@ -0,0 +1,21 @@ +16667 Tuple(a Dynamic(max_types=3)):Date +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):String +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 UInt64:None +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 UInt64:None +16667 Tuple(a Dynamic(max_types=3)):DateTime +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +133333 Tuple(a Dynamic(max_types=3)):None +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +116667 Tuple(a Dynamic(max_types=3)):String +133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh new file mode 100755 index 00000000000..f305fac071c --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" + +$CH_CLIENT -q "drop table if exists test;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" + +$CH_CLIENT -q "system stop merges test" +$CH_CLIENT -q "insert into test select number, number from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "drop table test;" + diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference new file mode 100644 index 00000000000..27ed336a035 --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference @@ -0,0 +1,21 @@ +16667 Tuple(a Dynamic(max_types=3)):Date +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):String +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 UInt64:None +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 UInt64:None +16667 Tuple(a Dynamic(max_types=3)):DateTime +33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) +50000 Tuple(a Dynamic(max_types=3)):UInt64 +66667 Tuple(a Dynamic(max_types=3)):String +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +133333 Tuple(a Dynamic(max_types=3)):None +50000 Tuple(a Dynamic(max_types=3)):UInt64 +100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 UInt64:None +116667 Tuple(a Dynamic(max_types=3)):String +133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh new file mode 100755 index 00000000000..0627244a02d --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" + +$CH_CLIENT -q "drop table if exists test;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" + +$CH_CLIENT -q "system stop merges test" +$CH_CLIENT -q "insert into test select number, number from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" +$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" + +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" +$CH_CLIENT -nm -q "system start merges test; optimize table test final;" +$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" + +$CH_CLIENT -q "drop table test;" + From d0053b505b7d05379644d5d0f47ec704d7496294 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 12:56:35 +0200 Subject: [PATCH 41/59] Rename test --- ...p.reference => 03208_keepermap_incomplete_data_drop.reference} | 0 ...led_data_drop.sql => 03208_keepermap_incomplete_data_drop.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03208_keepermap_failed_data_drop.reference => 03208_keepermap_incomplete_data_drop.reference} (100%) rename tests/queries/0_stateless/{03208_keepermap_failed_data_drop.sql => 03208_keepermap_incomplete_data_drop.sql} (100%) diff --git a/tests/queries/0_stateless/03208_keepermap_failed_data_drop.reference b/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.reference similarity index 100% rename from tests/queries/0_stateless/03208_keepermap_failed_data_drop.reference rename to tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.reference diff --git a/tests/queries/0_stateless/03208_keepermap_failed_data_drop.sql b/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql similarity index 100% rename from tests/queries/0_stateless/03208_keepermap_failed_data_drop.sql rename to tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql From 9f86b22d302c2feef5d666e75d11d09fe6ee1c6c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jul 2024 14:42:30 +0200 Subject: [PATCH 42/59] Speed up stateful tests table setup --- docker/test/stateful/run.sh | 6 +++--- docker/test/stress/run.sh | 6 +++--- tests/queries/1_stateful/00162_mmap_compression_none.sql | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 857385f4715..df85c047649 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -191,8 +191,8 @@ else ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" - clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" + clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" + clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" clickhouse-client --query "DROP TABLE datasets.visits_v1 SYNC" clickhouse-client --query "DROP TABLE datasets.hits_v1 SYNC" else @@ -200,7 +200,7 @@ else clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" fi clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0" + clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" fi clickhouse-client --query "SHOW TABLES FROM test" diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 86467394513..b21114e456f 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -209,9 +209,9 @@ clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDat ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='$TEMP_POLICY'" -clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" -clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" -clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" +clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" +clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" +clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" clickhouse-client --query "DROP TABLE datasets.visits_v1 SYNC" clickhouse-client --query "DROP TABLE datasets.hits_v1 SYNC" diff --git a/tests/queries/1_stateful/00162_mmap_compression_none.sql b/tests/queries/1_stateful/00162_mmap_compression_none.sql index d2cbcea8aaa..48d6ada821e 100644 --- a/tests/queries/1_stateful/00162_mmap_compression_none.sql +++ b/tests/queries/1_stateful/00162_mmap_compression_none.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS hits_none; CREATE TABLE hits_none (Title String CODEC(NONE)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -INSERT INTO hits_none SELECT Title FROM test.hits; +INSERT INTO hits_none SELECT Title FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16; SET min_bytes_to_use_mmap_io = 1; SELECT sum(length(Title)) FROM hits_none; From f5710beebf9394d954a1fe5dadb9df87b2b8ebaa Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Jul 2024 15:22:31 +0100 Subject: [PATCH 43/59] fix build --- src/Common/tests/gtest_cgroups_reader.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Common/tests/gtest_cgroups_reader.cpp b/src/Common/tests/gtest_cgroups_reader.cpp index 38e56401401..2de25bb42ce 100644 --- a/src/Common/tests/gtest_cgroups_reader.cpp +++ b/src/Common/tests/gtest_cgroups_reader.cpp @@ -1,12 +1,13 @@ +#if defined(OS_LINUX) + +#include + #include #include -#include -#include #include #include #include -#include "IO/WriteBufferFromFileBase.h" using namespace DB; @@ -173,3 +174,5 @@ INSTANTIATE_TEST_SUITE_P( CgroupsMemoryUsageObserverTests, CgroupsMemoryUsageObserverFixture, ::testing::Values(CgroupsMemoryUsageObserver::CgroupsVersion::V1, CgroupsMemoryUsageObserver::CgroupsVersion::V2)); + +#endif From 023276b6f6296a68f0e36004c9e40696bad73742 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 14:32:04 +0000 Subject: [PATCH 44/59] Fix tests reference --- .../03038_nested_dynamic_merges_compact_horizontal.reference | 4 ++-- .../03038_nested_dynamic_merges_compact_vertical.reference | 4 ++-- .../03038_nested_dynamic_merges_wide_horizontal.reference | 4 ++-- .../03038_nested_dynamic_merges_wide_vertical.reference | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference index 27ed336a035..4be740f6050 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference @@ -11,11 +11,11 @@ 33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) 50000 Tuple(a Dynamic(max_types=3)):UInt64 66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 133333 Tuple(a Dynamic(max_types=3)):None 50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 116667 Tuple(a Dynamic(max_types=3)):String 133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference index 27ed336a035..4be740f6050 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference @@ -11,11 +11,11 @@ 33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) 50000 Tuple(a Dynamic(max_types=3)):UInt64 66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 133333 Tuple(a Dynamic(max_types=3)):None 50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 116667 Tuple(a Dynamic(max_types=3)):String 133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference index 27ed336a035..4be740f6050 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference @@ -11,11 +11,11 @@ 33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) 50000 Tuple(a Dynamic(max_types=3)):UInt64 66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 133333 Tuple(a Dynamic(max_types=3)):None 50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 116667 Tuple(a Dynamic(max_types=3)):String 133333 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference index 27ed336a035..4be740f6050 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference @@ -11,11 +11,11 @@ 33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) 50000 Tuple(a Dynamic(max_types=3)):UInt64 66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 133333 Tuple(a Dynamic(max_types=3)):None 50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(number UInt64) +100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) 100000 UInt64:None 116667 Tuple(a Dynamic(max_types=3)):String 133333 Tuple(a Dynamic(max_types=3)):None From 1d7fcade729b68276dd3c58bec4d81983ca476b1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 17:28:12 +0200 Subject: [PATCH 45/59] Fixes --- src/Common/FailPoint.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 4 ++++ .../0_stateless/03208_keepermap_incomplete_data_drop.sql | 3 ++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index b952e9725e3..7b8b5036af0 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -58,7 +58,7 @@ static struct InitFiu PAUSEABLE(dummy_pausable_failpoint) \ ONCE(execute_query_calling_empty_set_result_func_on_exception) \ ONCE(receive_timeout_on_table_status_response) \ - ONCE(keepermap_fail_drop_data) \ + REGULAR(keepermap_fail_drop_data) \ namespace FailPoints diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 1c1de245d10..b32a2d302a7 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -417,6 +417,9 @@ StorageKeeperMap::StorageKeeperMap( auto code = client->tryCreate(zk_table_path, "", zkutil::CreateMode::Persistent); + /// A table on the same Keeper path already exists, we just appended our table id to subscribe as a new replica + /// We still don't know if the table matches the expected metadata so table_is_valid is not changed + /// It will be checked lazily on the first operation if (code == Coordination::Error::ZOK) return; @@ -474,6 +477,7 @@ StorageKeeperMap::StorageKeeperMap( table_is_valid = true; + /// we are the first table created for the specified Keeper path, i.e. we are the first replica return; } diff --git a/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql b/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql index ad0603f12a9..e8d4a5bc298 100644 --- a/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql +++ b/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql @@ -3,5 +3,6 @@ DROP TABLE IF EXISTS 03208_keepermap_test SYNC; CREATE TABLE 03208_keepermap_test (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test03208') PRIMARY KEY(key); INSERT INTO 03208_keepermap_test VALUES (1, 11); SYSTEM ENABLE FAILPOINT keepermap_fail_drop_data; -DROP TABLE 03208_keepermap_test; +DROP TABLE 03208_keepermap_test SYNC; -- { KEEPER_EXCEPTION } +SYSTEM DISABLE FAILPOINT keepermap_fail_drop_data; CREATE TABLE 03208_keepermap_test_another (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test03208') PRIMARY KEY(key); From 7353064d1c97231d02aab2ffade3e4c63a3ebcb1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 Jul 2024 23:12:22 +0200 Subject: [PATCH 46/59] fix --- tests/queries/0_stateless/replication.lib | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index b2fcc422508..fe867537000 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -52,7 +52,12 @@ function check_replication_consistency() check_query_part=$2 # Try to kill some mutations because sometimes tests run too much (it's not guarenteed to kill all mutations, see below) - ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null + # Try multiple replicas, because queries are not finished yet, and "global" KILL MUTATION may fail due to another query (like DROP TABLE) + readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%'") + for t in "${tables_arr[@]}" + do + ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table='$t'" > /dev/null 2>/dev/null + done # Wait for all queries to finish (query may still be running if thread is killed by timeout) num_tries=0 From abb6025e727d8b825ed62b07d8684dc0f7eddb7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 02:30:45 +0200 Subject: [PATCH 47/59] Fix performance test about the generateRandom table function, supposedly --- tests/performance/generate_table_function.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/generate_table_function.xml b/tests/performance/generate_table_function.xml index c219d73b6cf..d56c585188d 100644 --- a/tests/performance/generate_table_function.xml +++ b/tests/performance/generate_table_function.xml @@ -7,7 +7,7 @@ SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('d Date, dt DateTime, dtm DateTime(\'Asia/Istanbul\')', 0, 10, 10) LIMIT 1000000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Asia/Istanbul\')', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('f32 Float32, f64 Float64', 0, 10, 10) LIMIT 1000000000); - SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 0, 10, 10) LIMIT 1000000000); + SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Tuple(Int32, Int64)', 0, 10, 10) LIMIT 1000000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Int8)', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Nullable(Int32))', 0, 10, 10) LIMIT 100000000); From 4cbb8bab56c288a8a2d5bd5d0bc5ec102516c621 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 04:09:06 +0200 Subject: [PATCH 48/59] Speed up mutations for non-replicated MergeTree a bit --- src/Storages/StorageMergeTree.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 444b3fbae4c..45573925c02 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -506,13 +506,14 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context } Int64 version; + String mutation_id; { std::lock_guard lock(currently_processing_in_background_mutex); MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); version = increment.get(); entry.commit(version); - String mutation_id = entry.file_name; + mutation_id = entry.file_name; if (txn) txn->addMutation(shared_from_this(), mutation_id); @@ -527,9 +528,9 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context } throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); } - - LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } + + LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); background_operations_assignee.trigger(); return version; } From 8899d4242895893c0485efb2de237bca008d1a8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 04:29:46 +0200 Subject: [PATCH 49/59] Speed up mutations for non-replicated MergeTree significantly --- src/Storages/StorageMergeTree.cpp | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 45573925c02..527872d701e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -505,19 +505,18 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context additional_info = fmt::format(" (TID: {}; TIDH: {})", current_tid, current_tid.getHash()); } - Int64 version; - String mutation_id; + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); + Int64 version = increment.get(); + entry.commit(version); + String mutation_id = entry.file_name; + if (txn) + txn->addMutation(shared_from_this(), mutation_id); + + bool alter_conversions_mutations_updated = updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); + { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); - version = increment.get(); - entry.commit(version); - mutation_id = entry.file_name; - if (txn) - txn->addMutation(shared_from_this(), mutation_id); - - bool alter_conversions_mutations_updated = updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second; if (!inserted) { From 133e734d73b87707ccba17bd488cf60e1895abe5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 21:53:22 +0200 Subject: [PATCH 50/59] Move to integration test --- tests/integration/test_keeper_map/test.py | 21 +++++++++++++++++++ ...8_keepermap_incomplete_data_drop.reference | 0 .../03208_keepermap_incomplete_data_drop.sql | 8 ------- 3 files changed, 21 insertions(+), 8 deletions(-) delete mode 100644 tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.reference delete mode 100644 tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index d7b4230d872..31316af7b1e 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -104,3 +104,24 @@ def test_keeper_map_without_zk(started_cluster): node.query("DETACH TABLE test_keeper_map_without_zk") client.stop() + + +def test_keeper_map_with_failed_drop(started_cluster): + run_query( + "CREATE TABLE test_keeper_map_with_failed_drop (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_with_failed_drop') PRIMARY KEY(key);" + ) + + run_query("INSERT INTO test_keeper_map_with_failed_drop VALUES (1, 11)") + run_query("SYSTEM ENABLE FAILPOINT keepermap_fail_drop_data") + node.query("DROP TABLE test_keeper_map_with_failed_drop SYNC") + + zk_client = get_genuine_zk() + assert ( + zk_client.get("/test_keeper_map/test_keeper_map_with_failed_drop/data") + is not None + ) + + run_query("SYSTEM DISABLE FAILPOINT keepermap_fail_drop_data") + run_query( + "CREATE TABLE test_keeper_map_with_failed_drop_another (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_with_failed_drop') PRIMARY KEY(key);" + ) diff --git a/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.reference b/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql b/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql deleted file mode 100644 index e8d4a5bc298..00000000000 --- a/tests/queries/0_stateless/03208_keepermap_incomplete_data_drop.sql +++ /dev/null @@ -1,8 +0,0 @@ -DROP TABLE IF EXISTS 03208_keepermap_test SYNC; - -CREATE TABLE 03208_keepermap_test (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test03208') PRIMARY KEY(key); -INSERT INTO 03208_keepermap_test VALUES (1, 11); -SYSTEM ENABLE FAILPOINT keepermap_fail_drop_data; -DROP TABLE 03208_keepermap_test SYNC; -- { KEEPER_EXCEPTION } -SYSTEM DISABLE FAILPOINT keepermap_fail_drop_data; -CREATE TABLE 03208_keepermap_test_another (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test03208') PRIMARY KEY(key); From 9d7bb2a430b142e0761cc30efcf78ea1bbeb6871 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 23 Jul 2024 15:07:18 +0800 Subject: [PATCH 51/59] Move unit tests to gtest_convertFieldToType.cpp --- .../tests/gtest_convertFieldToType.cpp | 83 ++++++++++++++++++- src/Storages/Statistics/Statistics.cpp | 12 +-- .../Statistics/StatisticsCountMinSketch.cpp | 17 ++-- .../Statistics/StatisticsCountMinSketch.h | 4 +- src/Storages/Statistics/StatisticsTDigest.cpp | 6 +- src/Storages/Statistics/StatisticsTDigest.h | 4 +- src/Storages/Statistics/StatisticsUniq.cpp | 4 +- src/Storages/Statistics/StatisticsUniq.h | 4 +- src/Storages/Statistics/tests/gtest_stats.cpp | 47 ----------- 9 files changed, 109 insertions(+), 72 deletions(-) diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp index c8a9d5aa2c0..0cac9a3b59d 100644 --- a/src/Interpreters/tests/gtest_convertFieldToType.cpp +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -147,7 +147,7 @@ INSTANTIATE_TEST_SUITE_P( DecimalField(DateTime64(123 * Day * 1'000'000), 6) } }) - ); +); INSTANTIATE_TEST_SUITE_P( DateTimeToDateTime64, @@ -179,3 +179,84 @@ INSTANTIATE_TEST_SUITE_P( }, }) ); + +INSTANTIATE_TEST_SUITE_P( + StringToNumber, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + { + "String", + Field("1"), + "Int8", + Field(1) + }, + { + "String", + Field("256"), + "Int8", + Field() + }, + { + "String", + Field("not a number"), + "Int8", + {} + }, + { + "String", + Field("1.1"), + "Int8", + {} /// we can not convert '1.1' to Int8 + }, + { + "String", + Field("1.1"), + "Float64", + Field(1.1) + }, + }) +); + +INSTANTIATE_TEST_SUITE_P( + NumberToString, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + { + "Int8", + Field(1), + "String", + Field("1") + }, + { + "Int8", + Field(-1), + "String", + Field("-1") + }, + { + "Float64", + Field(1.1), + "String", + Field("1.1") + }, + }) +); + +INSTANTIATE_TEST_SUITE_P( + StringToDate, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + { + "String", + Field("2024-07-12"), + "Date", + Field(static_cast(19916)) + }, + { + "String", + Field("not a date"), + "Date", + {} + }, + }) +); diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 588e20e801f..ade3326288a 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -204,15 +204,15 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { - registerValidator(StatisticsType::TDigest, TDigestValidator); - registerCreator(StatisticsType::TDigest, TDigestCreator); + registerValidator(StatisticsType::TDigest, tdigestValidator); + registerCreator(StatisticsType::TDigest, tdigestCreator); - registerValidator(StatisticsType::Uniq, UniqValidator); - registerCreator(StatisticsType::Uniq, UniqCreator); + registerValidator(StatisticsType::Uniq, uniqValidator); + registerCreator(StatisticsType::Uniq, uniqCreator); #if USE_DATASKETCHES - registerValidator(StatisticsType::CountMinSketch, CountMinSketchValidator); - registerCreator(StatisticsType::CountMinSketch, CountMinSketchCreator); + registerValidator(StatisticsType::CountMinSketch, countMinSketchValidator); + registerCreator(StatisticsType::CountMinSketch, countMinSketchCreator); #endif } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index 95a8ceda8c8..e69bbc1515b 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -17,13 +17,18 @@ extern const int LOGICAL_ERROR; extern const int ILLEGAL_STATISTICS; } -/// Constants chosen based on rolling dices, which provides an error tolerance of 0.1% (ε = 0.001) and a confidence level of 99.9% (δ = 0.001). +/// Constants chosen based on rolling dices. +/// The values provides: +/// 1. an error tolerance of 0.1% (ε = 0.001) +/// 2. a confidence level of 99.9% (δ = 0.001). /// And sketch the size is 152kb. static constexpr auto num_hashes = 7uz; static constexpr auto num_buckets = 2718uz; StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_), sketch(num_hashes, num_buckets), data_type(data_type_) + : IStatistics(stat_) + , sketch(num_hashes, num_buckets) + , data_type(data_type_) { } @@ -50,8 +55,7 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const void StatisticsCountMinSketch::update(const ColumnPtr & column) { - size_t size = column->size(); - for (size_t row = 0; row < size; ++row) + for (size_t row = 0; row < column->size(); ++row) { if (column->isNullAt(row)) continue; @@ -80,16 +84,15 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) } -void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); - /// Data types of Numeric, String family, IPv4, IPv6, Date family, Enum family are supported. if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index aa71c643c05..6c8b74f8c35 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -31,8 +31,8 @@ private: }; -void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index e3a59f3251a..66150e00fdb 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -46,7 +46,7 @@ Float64 StatisticsTDigest::estimateLess(const Field & val) const auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); if (val_as_float) return t_digest.getCountLessThan(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimate value of type {}", val.getTypeName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const @@ -57,7 +57,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); } -void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -65,7 +65,7 @@ void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) { return std::make_shared(stat); } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 801d0787eaf..614973e5d8b 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -23,7 +23,7 @@ private: QuantileTDigest t_digest; }; -void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 9eea1dec39b..8f60ffcf0b5 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -60,7 +60,7 @@ void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index 5290585bd94..faabde8d47c 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -27,7 +27,7 @@ private: }; -void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); } diff --git a/src/Storages/Statistics/tests/gtest_stats.cpp b/src/Storages/Statistics/tests/gtest_stats.cpp index 9b9fae83109..e55c52c49f3 100644 --- a/src/Storages/Statistics/tests/gtest_stats.cpp +++ b/src/Storages/Statistics/tests/gtest_stats.cpp @@ -44,50 +44,3 @@ TEST(Statistics, TDigestLessThan) std::reverse(data.begin(), data.end()); test_less_than(data, {-1, 1e9, 50000.0, 3000.0, 30.0}, {0, 100000, 50000, 3000, 30}, {0, 0, 0.001, 0.001, 0.001}); } - -using Fields = std::vector; - -template -void testConvertFieldToDataType(const DataTypePtr & data_type, const Fields & fields, const T & expected_value, bool convert_failed = false) -{ - for (const auto & field : fields) - { - Field converted_value; - try - { - converted_value = convertFieldToType(field, *data_type); - } - catch(...) - { - ASSERT_TRUE(convert_failed); - } - if (convert_failed) - ASSERT_TRUE(converted_value.isNull()); - else - ASSERT_EQ(converted_value.template get(), expected_value); - } -} - -TEST(Statistics, convertFieldToType) -{ - Fields fields; - - auto data_type_int8 = DataTypeFactory::instance().get("Int8"); - fields = {1, 1.0, "1"}; - testConvertFieldToDataType(data_type_int8, fields, static_cast(1)); - - fields = {256, 1.1, "not a number"}; - testConvertFieldToDataType(data_type_int8, fields, static_cast(1), true); - - auto data_type_float64 = DataTypeFactory::instance().get("Float64"); - fields = {1, 1.0, "1.0"}; - testConvertFieldToDataType(data_type_float64, fields, static_cast(1.0)); - - auto data_type_string = DataTypeFactory::instance().get("String"); - fields = {1, "1"}; - testConvertFieldToDataType(data_type_string, fields, static_cast("1")); - - auto data_type_date = DataTypeFactory::instance().get("Date"); - fields = {"2024-07-12", 19916}; - testConvertFieldToDataType(data_type_date, fields, static_cast(19916)); -} From 2b56cbc1fdff42e31a9fee80f931726a5123e675 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 23 Jul 2024 09:42:58 +0200 Subject: [PATCH 52/59] CI: Add ec2 instance lifecycle metadata to CIDB --- tests/ci/clickhouse_helper.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 91ea5c6d5d3..287970cce9a 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -197,6 +197,10 @@ def get_instance_id(): return _query_imds("latest/meta-data/instance-id") +def get_instance_lifecycle(): + return _query_imds("latest/meta-data/instance-life-cycle") + + def prepare_tests_results_for_clickhouse( pr_info: PRInfo, test_results: TestResults, @@ -233,7 +237,7 @@ def prepare_tests_results_for_clickhouse( "head_ref": head_ref, "head_repo": head_repo, "task_url": pr_info.task_url, - "instance_type": get_instance_type(), + "instance_type": ",".join([get_instance_type(), get_instance_lifecycle()]), "instance_id": get_instance_id(), } From b164014bf99278f6e1bac67d3066d77aefc229c2 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 23 Jul 2024 11:00:05 +0200 Subject: [PATCH 53/59] CI: Remove ci runners scripts from oss --- tests/ci/worker/.gitignore | 1 - tests/ci/worker/deploy-runner-init.sh | 87 ------ tests/ci/worker/init_runner.sh | 406 -------------------------- 3 files changed, 494 deletions(-) delete mode 100644 tests/ci/worker/.gitignore delete mode 100755 tests/ci/worker/deploy-runner-init.sh delete mode 100644 tests/ci/worker/init_runner.sh diff --git a/tests/ci/worker/.gitignore b/tests/ci/worker/.gitignore deleted file mode 100644 index 4ed18989e78..00000000000 --- a/tests/ci/worker/.gitignore +++ /dev/null @@ -1 +0,0 @@ -generated_*init_runner.sh diff --git a/tests/ci/worker/deploy-runner-init.sh b/tests/ci/worker/deploy-runner-init.sh deleted file mode 100755 index 96fbd82a99c..00000000000 --- a/tests/ci/worker/deploy-runner-init.sh +++ /dev/null @@ -1,87 +0,0 @@ -#!/usr/bin/env bash - -set -e - -usage() { - echo "Usage: $0 ENVIRONMENT" >&2 - echo "Valid values for ENVIRONMENT: staging, production" >&2 - exit 1 -} - -case "$1" in - staging|production) - ENVIRONMENT="$1" ;; - --help) - usage ;; - *) - echo "Invalid argument" >&2 - usage ;; -esac - -cd "$(dirname "$0")" || exit 1 -SOURCE_SCRIPT='init_runner.sh' - -check_response() { - # Are we even in the interactive shell? - [ -t 1 ] || return 1 - local request - request="$1" - read -rp "$request (y/N): " response - case "$response" in - [Yy]) - return 0 - # Your code to continue goes here - ;; - *) - return 1 - ;; - esac -} - -check_dirty() { - if [ -n "$(git status --porcelain=v2 "$SOURCE_SCRIPT")" ]; then - echo "The $SOURCE_SCRIPT has uncommited changes, won't deploy it" >&2 - exit 1 - fi -} -GIT_HASH=$(git log -1 --format=format:%H) - -header() { - cat << EOF -#!/usr/bin/env bash - -echo 'The $ENVIRONMENT script is generated from $SOURCE_SCRIPT, commit $GIT_HASH' - -EOF -} - -body() { - local first_line - first_line=$(sed -n '/^# THE SCRIPT START$/{=;q;}' "$SOURCE_SCRIPT") - if [ -z "$first_line" ]; then - echo "The pattern '# THE SCRIPT START' is not found in $SOURCE_SCRIPT" >&2 - exit 1 - fi - tail "+$first_line" "$SOURCE_SCRIPT" -} - -GENERATED_FILE="generated_${ENVIRONMENT}_${SOURCE_SCRIPT}" - -{ header && body; } > "$GENERATED_FILE" - -echo "The file $GENERATED_FILE is generated" - -if check_response "Display the content of $GENERATED_FILE?"; then - if [ -z "$PAGER" ]; then - less "$GENERATED_FILE" - else - $PAGER "$GENERATED_FILE" - fi -fi - -check_dirty - -S3_OBJECT=${S3_OBJECT:-s3://github-runners-data/cloud-init/${ENVIRONMENT}.sh} -if check_response "Deploy the generated script to $S3_OBJECT?"; then - aws s3 mv "$GENERATED_FILE" "$S3_OBJECT" -fi diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh deleted file mode 100644 index 5177e112edd..00000000000 --- a/tests/ci/worker/init_runner.sh +++ /dev/null @@ -1,406 +0,0 @@ -#!/usr/bin/env bash - -cat > /dev/null << 'EOF' -The following content is embedded into the s3 object via the script -deploy-runner-init.sh {staging,production} -with additional helping information - -In the `user data` you should define as the following text -between `### COPY BELOW` and `### COPY ABOVE` - -### COPY BELOW -Content-Type: multipart/mixed; boundary="//" -MIME-Version: 1.0 - ---// -Content-Type: text/cloud-config; charset="us-ascii" -MIME-Version: 1.0 -Content-Transfer-Encoding: 7bit -Content-Disposition: attachment; filename="cloud-config.txt" - -#cloud-config -cloud_final_modules: -- [scripts-user, always] - ---// -Content-Type: text/x-shellscript; charset="us-ascii" -MIME-Version: 1.0 -Content-Transfer-Encoding: 7bit -Content-Disposition: attachment; filename="userdata.txt" - -#!/bin/bash -INSTANCE_ID=$(ec2metadata --instance-id) -INIT_ENVIRONMENT=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:init-environment'].Value" --output text) -echo "Downloading and using $INIT_ENVIRONMENT cloud-init.sh" -aws s3 cp "s3://github-runners-data/cloud-init/${INIT_ENVIRONMENT:-production}.sh" /tmp/cloud-init.sh -chmod 0700 /tmp/cloud-init.sh -exec bash /tmp/cloud-init.sh ---// -### COPY ABOVE -EOF - -# THE SCRIPT START - -set -uo pipefail - -#################################### -# IMPORTANT! # -# EC2 instance should have # -# `github:runner-type` tag # -# set accordingly to a runner role # -#################################### - -echo "Running init v1.1" -export DEBIAN_FRONTEND=noninteractive -export RUNNER_HOME=/home/ubuntu/actions-runner - -export RUNNER_ORG="ClickHouse" -export RUNNER_URL="https://github.com/${RUNNER_ORG}" -# Funny fact, but metadata service has fixed IP -INSTANCE_ID=$(ec2metadata --instance-id) -export INSTANCE_ID - -bash /usr/local/share/scripts/init-network.sh - -# combine labels -RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:runner-type'].Value" --output text) -LABELS="self-hosted,Linux,$(uname -m),$RUNNER_TYPE" -export LABELS -echo "Instance Labels: $LABELS" - -LIFE_CYCLE=$(curl -s --fail http://169.254.169.254/latest/meta-data/instance-life-cycle) -export LIFE_CYCLE -echo "Instance lifecycle: $LIFE_CYCLE" - -INSTANCE_TYPE=$(ec2metadata --instance-type) -echo "Instance type: $INSTANCE_TYPE" - -# Refresh CloudWatch agent config -aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json -systemctl restart amazon-cloudwatch-agent.service - -# Refresh teams ssh keys -TEAM_KEYS_URL=$(aws ssm get-parameter --region us-east-1 --name team-keys-url --query 'Parameter.Value' --output=text) -curl -s "${TEAM_KEYS_URL}" > /home/ubuntu/.ssh/authorized_keys2 -chown ubuntu: /home/ubuntu/.ssh -R - - -# Create a pre-run script that will provide diagnostics info -mkdir -p /tmp/actions-hooks -cat > /tmp/actions-hooks/common.sh << 'EOF' -#!/bin/bash -EOF - -terminate_delayed() { - # The function for post hook to gracefully finish the job and then tear down - # The very specific sleep time is used later to determine in the main loop if - # the instance is tearing down - # IF `sleep` IS CHANGED, CHANGE ANOTHER VALUE IN `pgrep` - sleep=13.14159265358979323846 - echo "Going to terminate the runner's instance in $sleep seconds" - # We execute it with `at` to not have it as an orphan process, but launched independently - # GH Runners kill all remain processes - echo "sleep '$sleep'; aws ec2 terminate-instances --instance-ids $INSTANCE_ID" | at now || \ - aws ec2 terminate-instances --instance-ids "$INSTANCE_ID" # workaround for complete out of space or non-installed `at` - exit 0 -} - -detect_delayed_termination() { - # The function look for very specific sleep with pi - if pgrep 'sleep 13.14159265358979323846'; then - echo 'The instance has delayed termination, sleep the same time to wait if it goes down' - sleep 14 - fi -} - -declare -f terminate_delayed >> /tmp/actions-hooks/common.sh - -terminate_and_exit() { - # Terminate instance and exit from the script instantly - echo "Going to terminate the runner's instance" - aws ec2 terminate-instances --instance-ids "$INSTANCE_ID" - exit 0 -} - -terminate_decrease_and_exit() { - # Terminate instance and exit from the script instantly - echo "Going to terminate the runner's instance and decrease asg capacity" - aws autoscaling terminate-instance-in-auto-scaling-group --instance-id "$INSTANCE_ID" --should-decrement-desired-capacity - exit 0 -} - -declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh - -check_spot_instance_is_old() { - if [ "$LIFE_CYCLE" == "spot" ]; then - local UPTIME - UPTIME=$(< /proc/uptime) - UPTIME=${UPTIME%%.*} - if (( 3600 < UPTIME )); then - echo "The spot instance has uptime $UPTIME, it's time to shut it down" - return 0 - fi - fi - return 1 -} - -check_proceed_spot_termination() { - # The function checks and proceeds spot instance termination if exists - # The event for spot instance termination - local FORCE - FORCE=${1:-} - if TERMINATION_DATA=$(curl -s --fail http://169.254.169.254/latest/meta-data/spot/instance-action); then - # https://docs.aws.amazon.com/AWSEC2/latest/UserGuide/spot-instance-termination-notices.html#instance-action-metadata - _action=$(jq '.action' -r <<< "$TERMINATION_DATA") - _time=$(jq '.time | fromdate' <<< "$TERMINATION_DATA") - _until_action=$((_time - $(date +%s))) - echo "Received the '$_action' event that will be effective in $_until_action seconds" - if (( _until_action <= 30 )) || [ "$FORCE" == "force" ]; then - echo "The action $_action will be done in $_until_action, killing the runner and exit" - local runner_pid - runner_pid=$(pgrep Runner.Listener) - if [ -n "$runner_pid" ]; then - # Kill the runner to not allow it cancelling the job - # shellcheck disable=SC2046 - kill -9 "$runner_pid" $(list_children "$runner_pid") - fi - sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" - terminate_and_exit - fi - fi -} - -no_terminating_metadata() { - # The function check that instance could continue work - # Returns 1 if any of termination events are received - - # The event for rebalance recommendation. Not strict, so we have some room to make a decision here - if curl -s --fail http://169.254.169.254/latest/meta-data/events/recommendations/rebalance; then - echo 'Received recommendation to rebalance, checking the uptime' - local UPTIME - UPTIME=$(< /proc/uptime) - UPTIME=${UPTIME%%.*} - # We don't shutdown the instances younger than 30m - if (( 1800 < UPTIME )); then - # To not shutdown everything at once, use the 66% to survive - if (( $((RANDOM % 3)) == 0 )); then - echo 'The instance is older than 30m and won the roulette' - return 1 - fi - echo 'The instance is older than 30m, but is not chosen for rebalance' - else - echo 'The instance is younger than 30m, do not shut it down' - fi - fi - - # Checks if the ASG in a lifecycle hook state - local ASG_STATUS - ASG_STATUS=$(curl -s http://169.254.169.254/latest/meta-data/autoscaling/target-lifecycle-state) - if [ "$ASG_STATUS" == "Terminated" ]; then - echo 'The instance in ASG status Terminating:Wait' - return 1 - fi -} - -terminate_on_event() { - # If there is a rebalance event, then the instance could die soon - # Let's don't wait for it and terminate proactively - if curl -s --fail http://169.254.169.254/latest/meta-data/events/recommendations/rebalance; then - terminate_and_exit - fi - - # Here we check if the autoscaling group marked the instance for termination, and it's wait for the job to finish - ASG_STATUS=$(curl -s http://169.254.169.254/latest/meta-data/autoscaling/target-lifecycle-state) - if [ "$ASG_STATUS" == "Terminated" ]; then - INSTANCE_ID=$(ec2metadata --instance-id) - ASG_NAME=$(aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='aws:autoscaling:groupName'].Value" --output text) - LIFECYCLE_HOOKS=$(aws autoscaling describe-lifecycle-hooks --auto-scaling-group-name "$ASG_NAME" --query "LifecycleHooks[].LifecycleHookName" --output text) - for LCH in $LIFECYCLE_HOOKS; do - aws autoscaling complete-lifecycle-action --lifecycle-action-result CONTINUE \ - --lifecycle-hook-name "$LCH" --auto-scaling-group-name "$ASG_NAME" \ - --instance-id "$INSTANCE_ID" - true # autoformat issue - done - echo 'The runner is marked as "Terminated" by the autoscaling group, we are terminating' - terminate_and_exit - fi -} - -cat > /tmp/actions-hooks/pre-run.sh << EOF -#!/bin/bash -set -uo pipefail - -echo "Runner's public DNS: $(ec2metadata --public-hostname)" -echo "Runner's labels: ${LABELS}" -echo "Runner's instance type: $(ec2metadata --instance-type)" -EOF - -# Create a post-run script that will restart docker daemon before the job started -cat > /tmp/actions-hooks/post-run.sh << 'EOF' -#!/bin/bash -set -xuo pipefail - -source /tmp/actions-hooks/common.sh - -# Free KiB, free percents -ROOT_STAT=($(df / | awk '/\// {print $4 " " int($4/$2 * 100)}')) -if [[ ${ROOT_STAT[0]} -lt 3000000 ]] || [[ ${ROOT_STAT[1]} -lt 5 ]]; then - echo "The runner has ${ROOT_STAT[0]}KiB and ${ROOT_STAT[1]}% of free space on /" - terminate_delayed -fi - -# shellcheck disable=SC2046 -docker ps --quiet | xargs --no-run-if-empty docker kill ||: -# shellcheck disable=SC2046 -docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - -# If we have hanged containers after the previous commands, than we have a hanged one -# and should restart the daemon -if [ "$(docker ps --all --quiet)" ]; then - # Systemd service of docker has StartLimitBurst=3 and StartLimitInterval=60s, - # that's why we try restarting it for long - for i in {1..25}; - do - sudo systemctl restart docker && break || sleep 5 - done - - for i in {1..10} - do - docker info && break || sleep 2 - done - # Last chance, otherwise we have to terminate poor instance - docker info 1>/dev/null || { echo Docker unable to start; terminate_delayed ; } -fi -EOF - -get_runner_token() { - /usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value -} - -is_job_assigned() { - local runner_pid - runner_pid=$(pgrep Runner.Listener) - if [ -z "$runner_pid" ]; then - # if runner has finished, it's fine - return 0 - fi - local log_file - log_file=$(lsof -p "$runner_pid" 2>/dev/null | grep -o "$RUNNER_HOME/_diag/Runner.*log") - if [ -z "$log_file" ]; then - # assume, the process is over or just started - return 0 - fi - # So far it's the only solid way to determine that the job is starting - grep -q 'Terminal] .* Running job:' "$log_file" \ - && return 0 \ - || return 1 -} - -list_children () { - local children - children=$(ps --ppid "$1" -o pid=) - if [ -z "$children" ]; then - return - fi - - for pid in $children; do - list_children "$pid" - done - echo "$children" -} - -# There's possibility that it fails because the runner's version is outdated, -# so after the first failure we'll try to launch it with enabled autoupdate. -# -# We'll fail and terminate after 10 consequent failures. -ATTEMPT=0 -# In `kill` 0 means "all processes in process group", -1 is "all but PID 1" -# We use `-2` to get an error -RUNNER_PID=-2 - -while true; do - # Does not send signal, but checks that the process $RUNNER_PID is running - if kill -0 -- $RUNNER_PID; then - ATTEMPT=0 - echo "Runner is working with pid $RUNNER_PID, checking the metadata in background" - check_proceed_spot_termination - - if ! is_job_assigned; then - RUNNER_AGE=$(( $(date +%s) - $(stat -c +%Y /proc/"$RUNNER_PID" 2>/dev/null || date +%s) )) - echo "The runner is launched $RUNNER_AGE seconds ago and still hasn't received a job" - if (( 60 < RUNNER_AGE )); then - echo "Attempt to delete the runner for a graceful shutdown" - sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" \ - || continue - echo "Runner didn't launch or have assigned jobs after ${RUNNER_AGE} seconds, shutting down" - terminate_decrease_and_exit - fi - fi - else - if [ "$RUNNER_PID" != "-2" ]; then - wait $RUNNER_PID \ - && echo "Runner with PID $RUNNER_PID successfully finished" \ - || echo "Attempt $((++ATTEMPT)) to start the runner" - fi - if (( ATTEMPT > 10 )); then - echo "The runner has failed to start after $ATTEMPT attempt. Give up and terminate it" - terminate_and_exit - fi - - cd $RUNNER_HOME || terminate_and_exit - detect_delayed_termination - # If runner is not active, check that it needs to terminate itself - echo "Checking if the instance suppose to terminate" - no_terminating_metadata || terminate_on_event - check_spot_instance_is_old && terminate_and_exit - check_proceed_spot_termination force - - echo "Going to configure runner" - token_args=(--token "$(get_runner_token)") - config_args=( - "${token_args[@]}" --url "$RUNNER_URL" - --ephemeral --unattended --replace --runnergroup Default - --labels "$LABELS" --work _work --name "$INSTANCE_ID" - ) - if (( ATTEMPT > 1 )); then - echo 'The runner failed to start at least once. Removing it and then configuring with autoupdate enabled.' - sudo -u ubuntu ./config.sh remove "${token_args[@]}" - sudo -u ubuntu ./config.sh "${config_args[@]}" - else - echo "Configure runner with disabled autoupdate" - config_args+=("--disableupdate") - sudo -u ubuntu ./config.sh "${config_args[@]}" - fi - - echo "Another one check to avoid race between runner and infrastructure" - no_terminating_metadata || terminate_on_event - check_spot_instance_is_old && terminate_and_exit - check_proceed_spot_termination force - - # There were some failures to start the Job because of trash in _work - rm -rf _work - - # https://github.com/actions/runner/issues/3266 - # We're unable to know if the runner is failed to start. - echo 'Monkey-patching run helpers to get genuine exit code of the runner' - for script in run.sh run-helper.sh.template; do - # shellcheck disable=SC2016 - grep -q 'exit 0$' "$script" && \ - sed 's/exit 0/exit $returnCode/' -i "$script" && \ - echo "Script $script is patched" - done - - echo "Run" - sudo -u ubuntu \ - ACTIONS_RUNNER_HOOK_JOB_STARTED=/tmp/actions-hooks/pre-run.sh \ - ACTIONS_RUNNER_HOOK_JOB_COMPLETED=/tmp/actions-hooks/post-run.sh \ - ./run.sh & - RUNNER_PID=$! - - sleep 10 - fi - - sleep 5 -done - -# vim:ts=4:sw=4 From bb28a65e9814302307c52d9e9b7ea6d052624fad Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 23 Jul 2024 11:25:19 +0200 Subject: [PATCH 54/59] minor fix for wf status reporting --- .github/workflows/backport_branches.yml | 3 --- .github/workflows/master.yml | 3 --- .github/workflows/merge_queue.yml | 3 --- .github/workflows/nightly.yml | 3 --- .github/workflows/pull_request.yml | 3 --- .github/workflows/release_branches.yml | 4 +--- tests/ci/ci_buddy.py | 1 + tests/ci/ci_utils.py | 23 +++++++++++++++++++---- 8 files changed, 21 insertions(+), 22 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 50f4f503f5d..c602a46d23c 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -272,7 +272,4 @@ jobs: cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF - echo "::group::Workflow results" - python3 -m json.tool "$WORKFLOW_RESULT_FILE" - echo "::endgroup::" python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index b28d87ee31f..7c319da6045 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -138,7 +138,4 @@ jobs: cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF - echo "::group::Workflow results" - python3 -m json.tool "$WORKFLOW_RESULT_FILE" - echo "::endgroup::" python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index db89825a99a..4b186241a0e 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -111,7 +111,4 @@ jobs: cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF - echo "::group::Workflow results" - python3 -m json.tool "$WORKFLOW_RESULT_FILE" - echo "::endgroup::" python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index fd5b5eefcc4..84db3338065 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -57,7 +57,4 @@ jobs: cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF - echo "::group::Workflow results" - python3 -m json.tool "$WORKFLOW_RESULT_FILE" - echo "::endgroup::" python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 9930cf6dde4..c7d7b28af38 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -171,9 +171,6 @@ jobs: cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF - echo "::group::Workflow results" - python3 -m json.tool "$WORKFLOW_RESULT_FILE" - echo "::endgroup::" python3 ./tests/ci/ci_buddy.py --check-wf-status ################################# Stage Final ################################# diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 50565112825..bca9ff33cd0 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -492,7 +492,5 @@ jobs: cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF - echo "::group::Workflow results" - python3 -m json.tool "$WORKFLOW_RESULT_FILE" - echo "::endgroup::" + python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index 688c7d59988..dfb5885270a 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -31,6 +31,7 @@ class CIBuddy: self.sha = pr_info.sha[:10] def check_workflow(self): + GHActions.print_workflow_results() res = GHActions.get_workflow_job_result(GHActions.ActionsNames.RunConfig) if res != GHActions.ActionStatuses.SUCCESS: self.post_job_error("Workflow Configuration Failed", critical=True) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 1963e3f39d0..3b12fe6974f 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -92,15 +92,30 @@ class GHActions: PENDING = "pending" SUCCESS = "success" - @staticmethod - def get_workflow_job_result(wf_job_name: str) -> Optional[str]: + @classmethod + def _get_workflow_results(cls): if not Path(Envs.WORKFLOW_RESULT_FILE).exists(): print( f"ERROR: Failed to get workflow results from file [{Envs.WORKFLOW_RESULT_FILE}]" ) - return None + return {} with open(Envs.WORKFLOW_RESULT_FILE, "r", encoding="utf-8") as json_file: - res = json.load(json_file) + try: + res = json.load(json_file) + except json.JSONDecodeError as e: + print(f"ERROR: json decoder exception {e}") + return {} + return res + + @classmethod + def print_workflow_results(cls): + res = cls._get_workflow_results() + results = [f"{job}: {data['result']}" for job, data in res.items()] + cls.print_in_group("Workflow results", results) + + @classmethod + def get_workflow_job_result(cls, wf_job_name: str) -> Optional[str]: + res = cls._get_workflow_results() if wf_job_name in res: return res[wf_job_name]["result"] # type: ignore else: From 53c7bf680b401dc4948353dec2d94a428327b1f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 Jul 2024 11:46:48 +0000 Subject: [PATCH 55/59] Update autogenerated version to 24.8.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 +++---- .../StorageSystemContributors.generated.cpp | 27 +++++++++++++++++++ 2 files changed, 32 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index bb776fa9506..d69646d3694 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54488) +SET(VERSION_REVISION 54489) SET(VERSION_MAJOR 24) -SET(VERSION_MINOR 7) +SET(VERSION_MINOR 8) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH aa023477a9265e403982fca5ee29a714db5133d9) -SET(VERSION_DESCRIBE v24.7.1.1-testing) -SET(VERSION_STRING 24.7.1.1) +SET(VERSION_GITHASH 3f8b27d7accd2b5ec4afe7d0dd459115323304af) +SET(VERSION_DESCRIBE v24.8.1.1-testing) +SET(VERSION_STRING 24.8.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 9201eef185f..35b9c0008c6 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -1,5 +1,6 @@ // autogenerated by tests/ci/version_helper.py const char * auto_contributors[] { + "0x01f", "0xflotus", "13DaGGeR", "1lann", @@ -167,6 +168,7 @@ const char * auto_contributors[] { "AnneClickHouse", "Anselmo D. Adams", "Anthony N. Simon", + "AntiTopQuark", "Anton Ivashkin", "Anton Kobzev", "Anton Kozlov", @@ -299,6 +301,7 @@ const char * auto_contributors[] { "Dan Wu", "DanRoscigno", "Dani Pozo", + "Daniel Anugerah", "Daniel Bershatsky", "Daniel Byta", "Daniel Dao", @@ -370,6 +373,7 @@ const char * auto_contributors[] { "Elena", "Elena Baskakova", "Elena Torró", + "Elena Torró Martínez", "Elghazal Ahmed", "Eliot Hautefeuille", "Elizaveta Mironyuk", @@ -415,6 +419,7 @@ const char * auto_contributors[] { "FgoDt", "Filatenkov Artur", "Filipe Caixeta", + "Filipp Bakanov", "Filipp Ozinov", "Filippov Denis", "Fille", @@ -451,6 +456,7 @@ const char * auto_contributors[] { "Gleb Novikov", "Gleb-Tretyakov", "GoGoWen2021", + "Gosha Letov", "Gregory", "Grigorii Sokolik", "Grigory", @@ -461,6 +467,7 @@ const char * auto_contributors[] { "Guillaume Tassery", "Guo Wangyang", "Guo Wei (William)", + "Guspan Tanadi", "Haavard Kvaalen", "Habibullah Oladepo", "HaiBo Li", @@ -474,6 +481,7 @@ const char * auto_contributors[] { "HarryLeeIBM", "Hasitha Kanchana", "Hasnat", + "Haydn", "Heena Bansal", "HeenaBansal2009", "Hendrik M", @@ -606,6 +614,7 @@ const char * auto_contributors[] { "Kevin Chiang", "Kevin Michel", "Kevin Mingtarja", + "Kevin Song", "Kevin Zhang", "KevinyhZou", "KinderRiven", @@ -661,6 +670,7 @@ const char * auto_contributors[] { "Lewinma", "Li Shuai", "Li Yin", + "Linh Giang", "Lino Uruñuela", "Lirikl", "Liu Cong", @@ -690,6 +700,7 @@ const char * auto_contributors[] { "Maksim Alekseev", "Maksim Buren", "Maksim Fedotov", + "Maksim Galkin", "Maksim Kita", "Maksym Sobolyev", "Mal Curtis", @@ -724,6 +735,7 @@ const char * auto_contributors[] { "Max Akhmedov", "Max Bruce", "Max K", + "Max K.", "Max Kainov", "Max Vetrov", "MaxTheHuman", @@ -811,6 +823,7 @@ const char * auto_contributors[] { "Nataly Merezhuk", "Natalya Chizhonkova", "Natasha Murashkina", + "Nathan Clevenger", "NeZeD [Mac Pro]", "Neeke Gao", "Neng Liu", @@ -946,6 +959,7 @@ const char * auto_contributors[] { "Robert Coelho", "Robert Hodges", "Robert Schulze", + "Rodolphe Dugé de Bernonville", "RogerYK", "Rohit Agarwal", "Romain Neutron", @@ -1107,6 +1121,7 @@ const char * auto_contributors[] { "Timur Solodovnikov", "TiunovNN", "Tobias Adamson", + "Tobias Florek", "Tobias Lins", "Tom Bombadil", "Tom Risse", @@ -1231,11 +1246,13 @@ const char * auto_contributors[] { "Yingchun Lai", "Yingfan Chen", "Yinzheng-Sun", + "Yinzuo Jiang", "Yiğit Konur", "Yohann Jardin", "Yong Wang", "Yong-Hao Zou", "Youenn Lebras", + "Your Name", "Yu, Peng", "Yuko Takagi", "Yuntao Wu", @@ -1250,6 +1267,7 @@ const char * auto_contributors[] { "Yury Stankevich", "Yusuke Tanaka", "Zach Naimon", + "Zawa-II", "Zheng Miao", "ZhiHong Zhang", "ZhiYong Wang", @@ -1380,6 +1398,7 @@ const char * auto_contributors[] { "conicliu", "copperybean", "coraxster", + "cw5121", "cwkyaoyao", "d.v.semenov", "dalei2019", @@ -1460,12 +1479,14 @@ const char * auto_contributors[] { "fuzzERot", "fyu", "g-arslan", + "gabrielmcg44", "ggerogery", "giordyb", "glockbender", "glushkovds", "grantovsky", "gulige", + "gun9nir", "guoleiyi", "guomaolin", "guov100", @@ -1527,6 +1548,7 @@ const char * auto_contributors[] { "jferroal", "jiahui-97", "jianmei zhang", + "jiaosenvip", "jinjunzh", "jiyoungyoooo", "jktng", @@ -1541,6 +1563,7 @@ const char * auto_contributors[] { "jun won", "jus1096", "justindeguzman", + "jwoodhead", "jyz0309", "karnevil13", "kashwy", @@ -1633,10 +1656,12 @@ const char * auto_contributors[] { "mateng0915", "mateng915", "mauidude", + "max-vostrikov", "maxim", "maxim-babenko", "maxkuzn", "maxulan", + "maxvostrikov", "mayamika", "mehanizm", "melin", @@ -1677,6 +1702,7 @@ const char * auto_contributors[] { "nathanbegbie", "nauta", "nautaa", + "nauu", "ndchikin", "nellicus", "nemonlou", @@ -1975,6 +2001,7 @@ const char * auto_contributors[] { "张健", "张风啸", "徐炘", + "忒休斯~Theseus", "曲正鹏", "木木夕120", "未来星___费", From c7427e6572f278d1a72289ca9ef50fbb701c1115 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 23 Jul 2024 16:36:30 +0200 Subject: [PATCH 56/59] CI: Fixes docker server build for release branches --- tests/ci/docker_server.py | 27 ++++++++++++--------------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 21fc02ce02a..413c35cbebe 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -21,7 +21,7 @@ from env_helper import ( TEMP_PATH, ) from git_helper import Git -from pr_info import PRInfo, EventType +from pr_info import PRInfo from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults from stopwatch import Stopwatch from tee_popen import TeePopen @@ -375,25 +375,23 @@ def main(): tags = gen_tags(args.version, args.release_type) repo_urls = {} direct_urls: Dict[str, List[str]] = {} - if pr_info.event_type == EventType.PULL_REQUEST: - release_or_pr = str(pr_info.number) - sha = pr_info.sha - elif pr_info.event_type == EventType.PUSH and pr_info.is_master: - release_or_pr = str(0) - sha = pr_info.sha - else: - release_or_pr = f"{args.version.major}.{args.version.minor}" - sha = args.sha - assert sha for arch, build_name in zip(ARCH, ("package_release", "package_aarch64")): - if not args.bucket_prefix: + if args.bucket_prefix: + assert not args.allow_build_reuse + repo_urls[arch] = f"{args.bucket_prefix}/{build_name}" + elif args.sha: + # CreateRelease workflow only. TODO + version = args.version repo_urls[arch] = ( f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/" - f"{release_or_pr}/{sha}/{build_name}" + f"{version.major}.{version.minor}/{args.sha}/{build_name}" ) else: - repo_urls[arch] = f"{args.bucket_prefix}/{build_name}" + # In all other cases urls must be fetched from build reports. TODO: script needs refactoring + repo_urls[arch] = "" + assert args.allow_build_reuse + if args.allow_build_reuse: # read s3 urls from pre-downloaded build reports if "clickhouse-server" in image_repo: @@ -431,7 +429,6 @@ def main(): ) if test_results[-1].status != "OK": status = FAILURE - pr_info = pr_info or PRInfo() description = f"Processed tags: {', '.join(tags)}" JobReport( From 2551f141418a9625678ea6295426bc1d10517482 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 23 Jul 2024 20:42:21 +0200 Subject: [PATCH 57/59] CI: Add messages fro debugging --- tests/ci/ci_utils.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 3b12fe6974f..d42091fb0da 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -104,6 +104,9 @@ class GHActions: res = json.load(json_file) except json.JSONDecodeError as e: print(f"ERROR: json decoder exception {e}") + json_file.seek(0) + print(" File content:") + print(json_file.read()) return {} return res From 8925060366e18e2fcb88d5d3c173097e524ce613 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Jul 2024 00:18:12 +0200 Subject: [PATCH 58/59] Touch dockerfile to forcefully rebuild image --- docker/test/sqlancer/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqlancer/Dockerfile b/docker/test/sqlancer/Dockerfile index 82fc2598397..3b919ffb3e3 100644 --- a/docker/test/sqlancer/Dockerfile +++ b/docker/test/sqlancer/Dockerfile @@ -6,7 +6,7 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list RUN apt-get update --yes \ - && env DEBIAN_FRONTEND=noninteractive apt-get install wget git default-jdk maven python3 --yes --no-install-recommends \ + && env DEBIAN_FRONTEND=noninteractive apt-get install wget git python3 default-jdk maven --yes --no-install-recommends \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* From e855ce793783bb50f95339608f688f6c27e83c36 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 04:59:57 +0200 Subject: [PATCH 59/59] Revert "Fix for 992 and friends" --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- ...00992_system_parts_race_condition_zookeeper_long.sh | 2 +- tests/queries/0_stateless/replication.lib | 10 +--------- 3 files changed, 3 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index da379a466af..3f02486ed15 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3940,7 +3940,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() merge_selecting_task->schedule(); else { - LOG_TRACE(log, "Scheduling next merge selecting task after {}ms, current attempt status: {}", merge_selecting_sleep_ms, result); + LOG_TRACE(log, "Scheduling next merge selecting task after {}ms", merge_selecting_sleep_ms); merge_selecting_task->scheduleAfter(merge_selecting_sleep_ms); } } diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 02a739ece4a..4887c409844 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -41,7 +41,7 @@ function thread3() function thread4() { - while true; do $CLICKHOUSE_CLIENT --receive_timeout=1 -q "OPTIMIZE TABLE alter_table0 FINAL" | grep -Fv "Timeout exceeded while receiving data from server"; done + while true; do $CLICKHOUSE_CLIENT --receive_timeout=3 -q "OPTIMIZE TABLE alter_table0 FINAL" | grep -Fv "Timeout exceeded while receiving data from server"; done } function thread5() diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index fe867537000..1a86cd9f8db 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -51,14 +51,6 @@ function check_replication_consistency() table_name_prefix=$1 check_query_part=$2 - # Try to kill some mutations because sometimes tests run too much (it's not guarenteed to kill all mutations, see below) - # Try multiple replicas, because queries are not finished yet, and "global" KILL MUTATION may fail due to another query (like DROP TABLE) - readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%'") - for t in "${tables_arr[@]}" - do - ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table='$t'" > /dev/null 2>/dev/null - done - # Wait for all queries to finish (query may still be running if thread is killed by timeout) num_tries=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%'") -ne 1 ]]; do @@ -104,7 +96,7 @@ function check_replication_consistency() some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1") $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA $some_table PULL" 1>/dev/null 2>/dev/null ||: - # Forcefully cancel mutations to avoid waiting for them to finish. Kills the remaining mutations + # Forcefully cancel mutations to avoid waiting for them to finish ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet