From 34d82625b58e157d937d48a9660079d63deab785 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 02:58:00 +0800 Subject: [PATCH 001/129] feat: add dotProduct for array --- src/Functions/array/arrayAUC.cpp | 2 +- src/Functions/array/arrayDotProduct.cpp | 75 ++++++++++++++++++++++++ src/Functions/array/arrayScalarProduct.h | 71 +++++++++++++--------- src/Functions/vectorFunctions.cpp | 16 ++++- 4 files changed, 133 insertions(+), 31 deletions(-) create mode 100644 src/Functions/array/arrayDotProduct.cpp diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 2890ae55886..297394822d9 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -91,7 +91,7 @@ public: return std::make_shared>(); } - template + template static ResultType apply( const T * scores, const U * labels, diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp new file mode 100644 index 00000000000..4b9433f683d --- /dev/null +++ b/src/Functions/array/arrayDotProduct.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include "arrayScalarProduct.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct NameArrayDotProduct +{ + static constexpr auto name = "arrayDotProduct"; +}; + + +class ArrayDotProductImpl +{ +public: + + static DataTypePtr getReturnType(const DataTypePtr left_type, const DataTypePtr & right_type) + { + const auto & common_type = getLeastSupertype(DataTypes{left_type, right_type}); + switch (common_type->getTypeId()) + { + case TypeIndex::UInt8: + case TypeIndex::UInt16: + case TypeIndex::UInt32: + case TypeIndex::Int8: + case TypeIndex::Int16: + case TypeIndex::Int32: + case TypeIndex::UInt64: + case TypeIndex::Int64: + case TypeIndex::Float64: + return std::make_shared(); + case TypeIndex::Float32: + return std::make_shared(); + default: + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} has nested type {}. " + "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + std::string(NameArrayDotProduct::name), + common_type->getName()); + } + } + + template + static ResultType apply( + const T * left, + const U * right, + size_t size) + { + ResultType result = 0; + for (size_t i = 0; i < size; ++i) + result += static_cast(left[i]) * static_cast(right[i]); + return result; + } + +}; + +using FunctionArrayDotProduct = FunctionArrayScalarProduct; + +REGISTER_FUNCTION(ArrayDotProduct) +{ + factory.registerFunction(); +} + +/// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp +FunctionPtr createFunctionArrayDotProduct(ContextPtr context_) { return FunctionArrayDotProduct::create(context_); } +} diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index 94ce1bc533c..ded6ec8ae29 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -29,29 +29,28 @@ public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: - using ResultColumnType = ColumnVector; - template + template ColumnPtr executeNumber(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if ( (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments))) + if ( (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments))) return res; return nullptr; } - template + template ColumnPtr executeNumberNumber(const ColumnsWithTypeAndName & arguments) const { ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); @@ -72,7 +71,7 @@ private: if (!col_nested1 || !col_nested2) return nullptr; - auto col_res = ResultColumnType::create(); + auto col_res = ColumnVector::create(); vector( col_nested1->getData(), @@ -83,12 +82,12 @@ private: return col_res; } - template + template static NO_INLINE void vector( const PaddedPODArray & data1, const PaddedPODArray & data2, const ColumnArray::Offsets & offsets, - PaddedPODArray & result) + PaddedPODArray & result) { size_t size = offsets.size(); result.resize(size); @@ -97,7 +96,7 @@ private: for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; - result[i] = Method::apply(&data1[current_offset], &data2[current_offset], array_size); + result[i] = Method::template apply(&data1[current_offset], &data2[current_offset], array_size); current_offset = offsets[i]; } } @@ -130,24 +129,40 @@ public: return Method::getReturnType(nested_types[0], nested_types[1]); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /* input_rows_count */) const override + template + ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if (!((res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)))) + if ( !((res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); return res; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override + { + switch (result_type->getTypeId()) + { + case TypeIndex::Float32: + return executeWithResultType(arguments); + break; + case TypeIndex::Float64: + return executeWithResultType(arguments); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName()); + } + } }; } diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index a7ab09612cf..b52def28755 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1429,6 +1429,8 @@ private: FunctionPtr array_function; }; +extern FunctionPtr createFunctionArrayDotProduct(ContextPtr context_); + extern FunctionPtr createFunctionArrayL1Norm(ContextPtr context_); extern FunctionPtr createFunctionArrayL2Norm(ContextPtr context_); extern FunctionPtr createFunctionArrayL2SquaredNorm(ContextPtr context_); @@ -1442,6 +1444,14 @@ extern FunctionPtr createFunctionArrayLpDistance(ContextPtr context_); extern FunctionPtr createFunctionArrayLinfDistance(ContextPtr context_); extern FunctionPtr createFunctionArrayCosineDistance(ContextPtr context_); +struct DotProduct +{ + static constexpr auto name = "dotProduct"; + + static constexpr auto CreateTupleFunction = FunctionDotProduct::create; + static constexpr auto CreateArrayFunction = createFunctionArrayDotProduct; +}; + struct L1NormTraits { static constexpr auto name = "L1Norm"; @@ -1530,6 +1540,8 @@ struct CosineDistanceTraits static constexpr auto CreateArrayFunction = createFunctionArrayCosineDistance; }; +using TupleOrArrayFunctionDotProduct = TupleOrArrayFunction; + using TupleOrArrayFunctionL1Norm = TupleOrArrayFunction; using TupleOrArrayFunctionL2Norm = TupleOrArrayFunction; using TupleOrArrayFunctionL2SquaredNorm = TupleOrArrayFunction; @@ -1615,8 +1627,8 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerAlias("scalarProduct", FunctionDotProduct::name, FunctionFactory::CaseInsensitive); + factory.registerFunction(); + factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); From 81fa4701aaa00974aeb8173c9847dfbe220c2a27 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 03:23:14 +0800 Subject: [PATCH 002/129] feat: add dotProduct for array --- .../0_stateless/02708_dot_product.reference | 10 ++++++ .../queries/0_stateless/02708_dot_product.sql | 34 +++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/02708_dot_product.reference create mode 100644 tests/queries/0_stateless/02708_dot_product.sql diff --git a/tests/queries/0_stateless/02708_dot_product.reference b/tests/queries/0_stateless/02708_dot_product.reference new file mode 100644 index 00000000000..7106b870fab --- /dev/null +++ b/tests/queries/0_stateless/02708_dot_product.reference @@ -0,0 +1,10 @@ +3881.304 +3881.304 +3881.304 +376.5 +230 +0 +Float64 +Float32 +Float64 +Float64 diff --git a/tests/queries/0_stateless/02708_dot_product.sql b/tests/queries/0_stateless/02708_dot_product.sql new file mode 100644 index 00000000000..46450ae6394 --- /dev/null +++ b/tests/queries/0_stateless/02708_dot_product.sql @@ -0,0 +1,34 @@ +SELECT dotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); +SELECT scalarProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); +SELECT arrayDotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); + +SELECT dotProduct([1.3, 2, 3, 4, 5], [222, 12, 5.3, 2, 8]); + +SELECT dotProduct([1, 1, 1, 1, 1], [222, 12, 0, -12, 8]); + +SELECT round(dotProduct([-1, 2, 3.002], [2, 3.4, 4]) - dotProduct((-1, 2, 3.002), (2, 3.4, 4)), 2); + + +DROP TABLE IF EXISTS product_fp64_fp64; +CREATE TABLE product_fp64_fp64 (x Array(Float64), y Array(Float64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_fp64_fp64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_fp64_fp64; +DROP TABLE product_fp64_fp64; + +DROP TABLE IF EXISTS product_fp32_fp32; +CREATE TABLE product_fp32_fp32 (x Array(Float32), y Array(Float32)) engine = MergeTree() order by x; +INSERT INTO TABLE product_fp32_fp32 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_fp32_fp32; +DROP TABLE product_fp32_fp32; + +DROP TABLE IF EXISTS product_fp32_fp64; +CREATE TABLE product_fp32_fp64 (x Array(Float32), y Array(Float64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_fp32_fp64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_fp32_fp64; +DROP TABLE product_fp32_fp64; + +DROP TABLE IF EXISTS product_uint8_fp64; +CREATE TABLE product_uint8_fp64 (x Array(UInt8), y Array(Float64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_uint8_fp64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_uint8_fp64; +DROP TABLE product_uint8_fp64; From 79ae949b566e5023500d037db6327a050dc7a11f Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 03:35:09 +0800 Subject: [PATCH 003/129] fix style --- src/Functions/array/arrayDotProduct.cpp | 7 ++----- src/Functions/array/arrayScalarProduct.h | 2 +- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 4b9433f683d..0d21bae90e3 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -17,12 +17,10 @@ struct NameArrayDotProduct static constexpr auto name = "arrayDotProduct"; }; - class ArrayDotProductImpl { public: - - static DataTypePtr getReturnType(const DataTypePtr left_type, const DataTypePtr & right_type) + static DataTypePtr getReturnType(const DataTypePtr & left_type, const DataTypePtr & right_type) { const auto & common_type = getLeastSupertype(DataTypes{left_type, right_type}); switch (common_type->getTypeId()) @@ -56,11 +54,10 @@ public: size_t size) { ResultType result = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < size; ++i) result += static_cast(left[i]) * static_cast(right[i]); return result; } - }; using FunctionArrayDotProduct = FunctionArrayScalarProduct; diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index ded6ec8ae29..0d1bf44a3e7 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -133,7 +133,7 @@ public: ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if ( !((res = executeNumber(arguments)) + if (!((res = executeNumber(arguments)) || (res = executeNumber(arguments)) || (res = executeNumber(arguments)) || (res = executeNumber(arguments)) From 4145abf547c3f09b849aa8e3429a318a340ef668 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 04:04:05 +0800 Subject: [PATCH 004/129] fix style --- src/Functions/array/arrayScalarProduct.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index 0d1bf44a3e7..5c36f2492c6 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } From 69e39aba80b4fadea79bfdf0f83a23848520948e Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 24 Apr 2023 22:13:13 +0800 Subject: [PATCH 005/129] fix test --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 4ff8c2d3af1..98cae995b47 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -97,6 +97,7 @@ arrayCumSum arrayCumSumNonNegative arrayDifference arrayDistinct +arrayDotProduct arrayElement arrayEnumerate arrayEnumerateDense From 913b63edc93c80f8bfaedfe2332859fc5dab83d3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 May 2023 21:28:33 +0200 Subject: [PATCH 006/129] Fix another zero copy bug --- src/Storages/MergeTree/MergeTreeData.cpp | 4 +-- src/Storages/MergeTree/ZeroCopyLock.cpp | 2 +- src/Storages/MergeTree/ZeroCopyLock.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 39 ++++++++++++++++++--- 4 files changed, 40 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fd98db7962e..e4181a5f9de 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7202,8 +7202,8 @@ std::pair MergeTreeData::cloneAn copy_instead_of_hardlink, files_to_copy_instead_of_hardlinks); - LOG_DEBUG(log, "Clone {} part {} to {}{}", - src_flushed_tmp_part ? "flushed" : "", + LOG_DEBUG(log, "Clone{} part {} to {}{}", + src_flushed_tmp_part ? " flushed" : "", src_part_storage->getFullPath(), std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), with_copy); diff --git a/src/Storages/MergeTree/ZeroCopyLock.cpp b/src/Storages/MergeTree/ZeroCopyLock.cpp index 53dfe0c769f..cca005dd7c0 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.cpp +++ b/src/Storages/MergeTree/ZeroCopyLock.cpp @@ -3,7 +3,7 @@ namespace DB { ZeroCopyLock::ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message) - : lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, "part_exclusive_lock", lock_message)) + : lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, ZERO_COPY_LOCK_NAME, lock_message)) { } } diff --git a/src/Storages/MergeTree/ZeroCopyLock.h b/src/Storages/MergeTree/ZeroCopyLock.h index 4400ea55b8f..2803952af18 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.h +++ b/src/Storages/MergeTree/ZeroCopyLock.h @@ -12,6 +12,8 @@ namespace DB /// because due to bad abstraction we use it in MergeTreeData. struct ZeroCopyLock { + static inline const std::string_view ZERO_COPY_LOCK_NAME = "part_exclusive_lock"; + ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message); bool isLocked() const { return lock->isLocked(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 20839a61c92..94abc1422fd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8249,7 +8249,7 @@ void StorageReplicatedMergeTree::lockSharedData( { String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; - LOG_TRACE(log, "Trying to create zookeeper persistent lock {}", zookeeper_node); + LOG_TRACE(log, "Trying to create zookeeper persistent lock {} with hardlinks [{}]", zookeeper_node, fmt::join(hardlinks, ", ")); createZeroCopyLockNode( zookeeper, zookeeper_node, zkutil::CreateMode::Persistent, @@ -8362,7 +8362,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair> getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, Poco::Logger * log) { NameSet files_not_to_remove; @@ -8404,15 +8404,40 @@ std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionP /// Get hardlinked files String files_not_to_remove_str; Coordination::Error code; - zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); + zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, &code); if (code != Coordination::Error::ZOK) + { LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + return {true, std::nullopt}; + } if (!files_not_to_remove_str.empty()) { boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); LOG_TRACE(log, "Found files not to remove from parent part {}: [{}]", part_candidate_info_str, fmt::join(files_not_to_remove, ", ")); } + else + { + std::vector children; + code = zookeeper_ptr->tryGetChildren(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, children); + if (code != Coordination::Error::ZOK) + { + LOG_TRACE(log, "Cannot get parent locks in ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + return {true, std::nullopt}; + } + + if (children.size() > 1 || children.size() == 1 && children[0] != ZeroCopyLock::ZERO_COPY_LOCK_NAME) + { + LOG_TRACE(log, "No files not to remove found for part {} from parent {}", part_info_str, part_candidate_info_str); + } + else + { + /// The case when part is actually removed, but some stale replica trying to execute merge/mutation. + /// We shouldn't use the part to check hardlinked blobs, it just doesn't exist. + LOG_TRACE(log, "Part {} is not parent (only merge/mutation locks exist), refusing to use as parent", part_candidate_info_str); + continue; + } + } return {true, files_not_to_remove}; } @@ -8448,6 +8473,12 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_info, data_format_version, logger); + if (has_parent && parent_not_to_remove == std::nullopt) + { + LOG_TRACE(logger, "Failed to get mutation parent on {} for part {}, refusing to remove blobs", zookeeper_part_replica_node, part_name); + return {false, {}}; + } + files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; @@ -8527,7 +8558,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (error_code == Coordination::Error::ZOK) { - LOG_TRACE(logger, "Removed last parent zookeeper lock {} for part {} (part is finally unlocked)", zookeeper_part_uniq_node, part_name); + LOG_TRACE(logger, "Removed last parent zookeeper lock {} for part {} (part is finally unlocked)", zookeeper_part_node, part_name); } else if (error_code == Coordination::Error::ZNOTEMPTY) { From 610e63bfd2b21e06c312c625e54279c31d06853c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 May 2023 23:54:24 +0200 Subject: [PATCH 007/129] Fix build --- src/Storages/MergeTree/ZeroCopyLock.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++++++++-------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ZeroCopyLock.h b/src/Storages/MergeTree/ZeroCopyLock.h index 2803952af18..d4c829a3652 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.h +++ b/src/Storages/MergeTree/ZeroCopyLock.h @@ -12,7 +12,7 @@ namespace DB /// because due to bad abstraction we use it in MergeTreeData. struct ZeroCopyLock { - static inline const std::string_view ZERO_COPY_LOCK_NAME = "part_exclusive_lock"; + static inline const auto ZERO_COPY_LOCK_NAME = "part_exclusive_lock"; ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 94abc1422fd..280150f27ad 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8404,7 +8404,7 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith /// Get hardlinked files String files_not_to_remove_str; Coordination::Error code; - zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, &code); + zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); if (code != Coordination::Error::ZOK) { LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); @@ -8426,7 +8426,7 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith return {true, std::nullopt}; } - if (children.size() > 1 || children.size() == 1 && children[0] != ZeroCopyLock::ZERO_COPY_LOCK_NAME) + if (children.size() > 1 || (children.size() == 1 && children[0] != ZeroCopyLock::ZERO_COPY_LOCK_NAME)) { LOG_TRACE(log, "No files not to remove found for part {} from parent {}", part_info_str, part_candidate_info_str); } @@ -8471,20 +8471,23 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!files_not_to_remove_str.empty()) boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); + String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; + + /// Delete our replica node for part from zookeeper (we are not interested in it anymore) + String zookeeper_part_replica_node = fs::path(zookeeper_part_uniq_node) / replica_name_; + auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_info, data_format_version, logger); - if (has_parent && parent_not_to_remove == std::nullopt) + + // parent_not_to_remove == std::nullopt means that we were unable to retrieve parts set + if (has_parent || parent_not_to_remove == std::nullopt) { LOG_TRACE(logger, "Failed to get mutation parent on {} for part {}, refusing to remove blobs", zookeeper_part_replica_node, part_name); return {false, {}}; } - files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); + files_not_to_remove.insert(parent_not_to_remove->begin(), parent_not_to_remove->end()); - String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; - - /// Delete our replica node for part from zookeeper (we are not interested in it anymore) - String zookeeper_part_replica_node = fs::path(zookeeper_part_uniq_node) / replica_name_; LOG_TRACE(logger, "Remove zookeeper lock {} for part {}", zookeeper_part_replica_node, part_name); From 3aaea2b91d34a98651e62f9cedd263d9333530d5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 4 May 2023 13:07:28 +0000 Subject: [PATCH 008/129] join thread even exception happened --- src/Interpreters/TraceCollector.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 49588d490f5..bedc27f7d15 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -56,9 +56,17 @@ TraceCollector::~TraceCollector() */ void TraceCollector::stop() { - WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); - writeChar(true, out); - out.next(); + try + { + WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); + writeChar(true, out); + out.next(); + } + catch (...) + { + tryLogCurrentException("TraceCollector"); + } + thread.join(); } From 1e3cd4ed6d39961addf82ab75d5fb2f661801ae5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 8 May 2023 12:39:01 +0000 Subject: [PATCH 009/129] close pipe before join --- src/Interpreters/TraceCollector.cpp | 47 ++++++++++++++--------------- src/Interpreters/TraceCollector.h | 1 - 2 files changed, 22 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index bedc27f7d15..0fc9aeb170b 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -34,40 +34,37 @@ TraceCollector::~TraceCollector() { try { - if (!thread.joinable()) - LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); + if (thread.joinable()) + { + /** Sends TraceCollector stop message + * + * Each sequence of data for TraceCollector thread starts with a boolean flag. + * If this flag is true, TraceCollector must stop reading trace_pipe and exit. + * This function sends flag with a true value to stop TraceCollector gracefully. + */ + WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); + writeChar(true, out); + out.next(); + } else - stop(); + LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); + } + catch (...) + { + tryLogCurrentException("TraceCollector"); + } + try + { TraceSender::pipe.close(); } catch (...) { tryLogCurrentException("TraceCollector"); } -} - -/** Sends TraceCollector stop message - * - * Each sequence of data for TraceCollector thread starts with a boolean flag. - * If this flag is true, TraceCollector must stop reading trace_pipe and exit. - * This function sends flag with a true value to stop TraceCollector gracefully. - */ -void TraceCollector::stop() -{ - try - { - WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); - writeChar(true, out); - out.next(); - } - catch (...) - { - tryLogCurrentException("TraceCollector"); - } - - thread.join(); + if (thread.joinable()) + thread.join(); } diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index 40fa854b791..d5bd09f1757 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -24,7 +24,6 @@ private: ThreadFromGlobalPool thread; void run(); - void stop(); }; } From 70516f2f451fd9bdb82f6e3ee689772fce589a8e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 8 May 2023 23:00:20 +0000 Subject: [PATCH 010/129] close pipe in run() if exception happens --- src/Interpreters/TraceCollector.cpp | 128 +++++++++++++++------------- src/Interpreters/TraceCollector.h | 2 + 2 files changed, 72 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 0fc9aeb170b..b8ef7044781 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -29,6 +29,17 @@ TraceCollector::TraceCollector(std::shared_ptr trace_log_) thread = ThreadFromGlobalPool(&TraceCollector::run, this); } +void TraceCollector::tryClosePipe() +{ + try + { + TraceSender::pipe.close(); + } + catch (...) + { + tryLogCurrentException("TraceCollector"); + } +} TraceCollector::~TraceCollector() { @@ -54,14 +65,7 @@ TraceCollector::~TraceCollector() tryLogCurrentException("TraceCollector"); } - try - { - TraceSender::pipe.close(); - } - catch (...) - { - tryLogCurrentException("TraceCollector"); - } + tryClosePipe(); if (thread.joinable()) thread.join(); @@ -74,60 +78,68 @@ void TraceCollector::run() ReadBufferFromFileDescriptor in(TraceSender::pipe.fds_rw[0]); - while (true) + try { - char is_last; - readChar(is_last, in); - if (is_last) - break; - - std::string query_id; - UInt8 query_id_size = 0; - readBinary(query_id_size, in); - query_id.resize(query_id_size); - in.readStrict(query_id.data(), query_id_size); - - UInt8 trace_size = 0; - readIntBinary(trace_size, in); - - Array trace; - trace.reserve(trace_size); - - for (size_t i = 0; i < trace_size; ++i) + while (true) { - uintptr_t addr = 0; - readPODBinary(addr, in); - trace.emplace_back(static_cast(addr)); - } - - TraceType trace_type; - readPODBinary(trace_type, in); - - UInt64 thread_id; - readPODBinary(thread_id, in); - - Int64 size; - readPODBinary(size, in); - - ProfileEvents::Event event; - readPODBinary(event, in); - - ProfileEvents::Count increment; - readPODBinary(increment, in); - - if (trace_log) - { - // time and time_in_microseconds are both being constructed from the same timespec so that the - // times will be equal up to the precision of a second. - struct timespec ts; - clock_gettime(CLOCK_REALTIME, &ts); - - UInt64 time = static_cast(ts.tv_sec * 1000000000LL + ts.tv_nsec); - UInt64 time_in_microseconds = static_cast((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); - TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment}; - trace_log->add(element); + char is_last; + readChar(is_last, in); + if (is_last) + break; + + std::string query_id; + UInt8 query_id_size = 0; + readBinary(query_id_size, in); + query_id.resize(query_id_size); + in.readStrict(query_id.data(), query_id_size); + + UInt8 trace_size = 0; + readIntBinary(trace_size, in); + + Array trace; + trace.reserve(trace_size); + + for (size_t i = 0; i < trace_size; ++i) + { + uintptr_t addr = 0; + readPODBinary(addr, in); + trace.emplace_back(static_cast(addr)); + } + + TraceType trace_type; + readPODBinary(trace_type, in); + + UInt64 thread_id; + readPODBinary(thread_id, in); + + Int64 size; + readPODBinary(size, in); + + ProfileEvents::Event event; + readPODBinary(event, in); + + ProfileEvents::Count increment; + readPODBinary(increment, in); + + if (trace_log) + { + // time and time_in_microseconds are both being constructed from the same timespec so that the + // times will be equal up to the precision of a second. + struct timespec ts; + clock_gettime(CLOCK_REALTIME, &ts); + + UInt64 time = static_cast(ts.tv_sec * 1000000000LL + ts.tv_nsec); + UInt64 time_in_microseconds = static_cast((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); + TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment}; + trace_log->add(element); + } } } + catch (...) + { + tryClosePipe(); + throw; + } } } diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index d5bd09f1757..382e7511ac6 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -23,6 +23,8 @@ private: std::shared_ptr trace_log; ThreadFromGlobalPool thread; + void tryClosePipe(); + void run(); }; From a7e04b7576d5dc5cc2b7feb4a043eb4361d9eeb9 Mon Sep 17 00:00:00 2001 From: fhbai Date: Tue, 9 May 2023 11:36:15 +0800 Subject: [PATCH 011/129] fix return type --- src/Functions/array/arrayDotProduct.cpp | 59 +++++++++++-------- src/Functions/array/arrayScalarProduct.h | 22 +++++-- .../0_stateless/02708_dot_product.reference | 4 ++ .../queries/0_stateless/02708_dot_product.sql | 23 +++++++- 4 files changed, 76 insertions(+), 32 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 0d21bae90e3..7aa9f1d49c7 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -1,7 +1,12 @@ #include #include #include -#include "arrayScalarProduct.h" +#include +#include +#include +#include +#include +#include namespace DB @@ -20,31 +25,33 @@ struct NameArrayDotProduct class ArrayDotProductImpl { public: - static DataTypePtr getReturnType(const DataTypePtr & left_type, const DataTypePtr & right_type) + static DataTypePtr getReturnType(const DataTypePtr & left, const DataTypePtr & right) { - const auto & common_type = getLeastSupertype(DataTypes{left_type, right_type}); - switch (common_type->getTypeId()) - { - case TypeIndex::UInt8: - case TypeIndex::UInt16: - case TypeIndex::UInt32: - case TypeIndex::Int8: - case TypeIndex::Int16: - case TypeIndex::Int32: - case TypeIndex::UInt64: - case TypeIndex::Int64: - case TypeIndex::Float64: - return std::make_shared(); - case TypeIndex::Float32: - return std::make_shared(); - default: - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", - std::string(NameArrayDotProduct::name), - common_type->getName()); - } + using Types = TypeList; + + DataTypePtr result_type; + bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_) { + return castTypeToEither(Types{}, right.get(), [&](const auto & right_) { + using LeftDataType = typename std::decay_t::FieldType; + using RightDataType = typename std::decay_t::FieldType; + using ResultType = typename NumberTraits::ResultOfAdditionMultiplication::Type; + if (std::is_same_v && std::is_same_v) + result_type = std::make_shared(); + else + result_type = std::make_shared>(); + return true; + }); + }); + + if (!valid) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} " + "only support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + std::string(NameArrayDotProduct::name)); + return result_type; } template @@ -67,6 +74,6 @@ REGISTER_FUNCTION(ArrayDotProduct) factory.registerFunction(); } -/// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp +// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp FunctionPtr createFunctionArrayDotProduct(ContextPtr context_) { return FunctionArrayDotProduct::create(context_); } } diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index 5c36f2492c6..374a2d8a194 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -154,12 +155,23 @@ public: { switch (result_type->getTypeId()) { - case TypeIndex::Float32: - return executeWithResultType(arguments); - break; - case TypeIndex::Float64: - return executeWithResultType(arguments); + #define SUPPORTED_TYPE(type) \ + case TypeIndex::type: \ + return executeWithResultType(arguments); \ break; + + SUPPORTED_TYPE(UInt8) + SUPPORTED_TYPE(UInt16) + SUPPORTED_TYPE(UInt32) + SUPPORTED_TYPE(UInt64) + SUPPORTED_TYPE(Int8) + SUPPORTED_TYPE(Int16) + SUPPORTED_TYPE(Int32) + SUPPORTED_TYPE(Int64) + SUPPORTED_TYPE(Float32) + SUPPORTED_TYPE(Float64) + #undef SUPPORTED_TYPE + default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName()); } diff --git a/tests/queries/0_stateless/02708_dot_product.reference b/tests/queries/0_stateless/02708_dot_product.reference index 7106b870fab..45e53871aa2 100644 --- a/tests/queries/0_stateless/02708_dot_product.reference +++ b/tests/queries/0_stateless/02708_dot_product.reference @@ -4,7 +4,11 @@ 376.5 230 0 +0 Float64 Float32 Float64 Float64 +UInt16 +UInt64 +Int64 diff --git a/tests/queries/0_stateless/02708_dot_product.sql b/tests/queries/0_stateless/02708_dot_product.sql index 46450ae6394..e94cb577bf4 100644 --- a/tests/queries/0_stateless/02708_dot_product.sql +++ b/tests/queries/0_stateless/02708_dot_product.sql @@ -1,13 +1,16 @@ SELECT dotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); + SELECT scalarProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); + SELECT arrayDotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); SELECT dotProduct([1.3, 2, 3, 4, 5], [222, 12, 5.3, 2, 8]); SELECT dotProduct([1, 1, 1, 1, 1], [222, 12, 0, -12, 8]); -SELECT round(dotProduct([-1, 2, 3.002], [2, 3.4, 4]) - dotProduct((-1, 2, 3.002), (2, 3.4, 4)), 2); +SELECT round(dotProduct([12345678901234567], [1]) - dotProduct(tuple(12345678901234567), tuple(1)), 2); +SELECT round(dotProduct([-1, 2, 3.002], [2, 3.4, 4]) - dotProduct((-1, 2, 3.002), (2, 3.4, 4)), 2); DROP TABLE IF EXISTS product_fp64_fp64; CREATE TABLE product_fp64_fp64 (x Array(Float64), y Array(Float64)) engine = MergeTree() order by x; @@ -32,3 +35,21 @@ CREATE TABLE product_uint8_fp64 (x Array(UInt8), y Array(Float64)) engine = Merg INSERT INTO TABLE product_uint8_fp64 (x, y) values ([1, 2], [3, 4]); SELECT toTypeName(dotProduct(x, y)) from product_uint8_fp64; DROP TABLE product_uint8_fp64; + +DROP TABLE IF EXISTS product_uint8_uint8; +CREATE TABLE product_uint8_uint8 (x Array(UInt8), y Array(UInt8)) engine = MergeTree() order by x; +INSERT INTO TABLE product_uint8_uint8 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_uint8_uint8; +DROP TABLE product_uint8_uint8; + +DROP TABLE IF EXISTS product_uint64_uint64; +CREATE TABLE product_uint64_uint64 (x Array(UInt64), y Array(UInt64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_uint64_uint64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_uint64_uint64; +DROP TABLE product_uint64_uint64; + +DROP TABLE IF EXISTS product_int32_uint64; +CREATE TABLE product_int32_uint64 (x Array(Int32), y Array(UInt64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_int32_uint64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_int32_uint64; +DROP TABLE product_int32_uint64; From 79398f612f4a94cee996d6256155c5e229ee090b Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Tue, 9 May 2023 11:50:38 +0800 Subject: [PATCH 012/129] fix style --- src/Functions/array/arrayDotProduct.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 7aa9f1d49c7..e3c80775f1b 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -32,8 +32,10 @@ public: DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64>; DataTypePtr result_type; - bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_) { - return castTypeToEither(Types{}, right.get(), [&](const auto & right_) { + bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_) + { + return castTypeToEither(Types{}, right.get(), [&](const auto & right_) + { using LeftDataType = typename std::decay_t::FieldType; using RightDataType = typename std::decay_t::FieldType; using ResultType = typename NumberTraits::ResultOfAdditionMultiplication::Type; From 815cc8de9bb843d6b0d54b5507f08533f52b8190 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 9 May 2023 19:43:39 +0000 Subject: [PATCH 013/129] removed unnecessary check for thread's joinability --- src/Interpreters/TraceCollector.cpp | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index b8ef7044781..cb00e37df69 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -45,20 +45,15 @@ TraceCollector::~TraceCollector() { try { - if (thread.joinable()) - { - /** Sends TraceCollector stop message - * - * Each sequence of data for TraceCollector thread starts with a boolean flag. - * If this flag is true, TraceCollector must stop reading trace_pipe and exit. - * This function sends flag with a true value to stop TraceCollector gracefully. - */ - WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); - writeChar(true, out); - out.next(); - } - else - LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); + /** Sends TraceCollector stop message + * + * Each sequence of data for TraceCollector thread starts with a boolean flag. + * If this flag is true, TraceCollector must stop reading trace_pipe and exit. + * This function sends flag with a true value to stop TraceCollector gracefully. + */ + WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); + writeChar(true, out); + out.next(); } catch (...) { @@ -69,6 +64,8 @@ TraceCollector::~TraceCollector() if (thread.joinable()) thread.join(); + else + LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); } From 0f7c0859b036ee1d9ede3da0fa0f325f7314364a Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 12 May 2023 09:22:29 -0400 Subject: [PATCH 014/129] add allow_experimental_parallel_reading_from_replicas docs --- docs/en/operations/settings/settings.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c1d6416837d..c560b8e2ad8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1366,6 +1366,12 @@ Possible values: Default value: `default`. +## allow_experimental_parallel_reading_from_replicas + +If true, ClickHouse will send a SELECT query to all replicas of a table (up to `max_parallel_replicas`) . It will work for any kind of MergeTree table. + +Default value: `false`. + ## compile_expressions {#compile-expressions} Enables or disables compilation of frequently used simple functions and operators to native code with LLVM at runtime. From 224646698e2748baaf732f1d9b59e61450d9d56b Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 12 May 2023 09:30:06 -0400 Subject: [PATCH 015/129] update settings header --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 215e5b38056..99e48a5b411 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -156,7 +156,7 @@ class IColumn; M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ - M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ + M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table (up to `max_parallel_replicas`). It will work for any kind of MergeTree table.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ \ From d2e185d86df4f7e6a7b19134d7de2f960e61a644 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 12 May 2023 14:36:54 +0000 Subject: [PATCH 016/129] Analyzer: do not optimize GROUP BY keys with ROLLUP and CUBE --- src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp | 3 +++ .../queries/0_stateless/02734_optimize_group_by.reference | 8 ++++++++ tests/queries/0_stateless/02734_optimize_group_by.sql | 7 +++++++ 3 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02734_optimize_group_by.reference create mode 100644 tests/queries/0_stateless/02734_optimize_group_by.sql diff --git a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp index c97645219da..5ed52f1210b 100644 --- a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp @@ -38,6 +38,9 @@ public: if (!query->hasGroupBy()) return; + if (query->isGroupByWithCube() || query->isGroupByWithRollup()) + return; + auto & group_by = query->getGroupBy().getNodes(); if (query->isGroupByWithGroupingSets()) { diff --git a/tests/queries/0_stateless/02734_optimize_group_by.reference b/tests/queries/0_stateless/02734_optimize_group_by.reference new file mode 100644 index 00000000000..3f5ef03cb61 --- /dev/null +++ b/tests/queries/0_stateless/02734_optimize_group_by.reference @@ -0,0 +1,8 @@ +a b +a b +a b + b +a b +a b +a b +a b diff --git a/tests/queries/0_stateless/02734_optimize_group_by.sql b/tests/queries/0_stateless/02734_optimize_group_by.sql new file mode 100644 index 00000000000..28e86c04b0f --- /dev/null +++ b/tests/queries/0_stateless/02734_optimize_group_by.sql @@ -0,0 +1,7 @@ +SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS allow_experimental_analyzer = 0; +SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS allow_experimental_analyzer = 1; + +SELECT 'a' AS key, 'b' as value GROUP BY ignore(1) WITH CUBE; + +SELECT 'a' AS key, 'b' as value GROUP BY ignore(1); +SELECT 'a' AS key, 'b' as value GROUP BY key; From d3c408aaf16356f59f91e95dac0b628e858f4640 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 May 2023 16:32:23 +0000 Subject: [PATCH 017/129] ClearableHashSet w/o zero storage for StringRef --- src/Common/HashTable/ClearableHashSet.h | 38 ----------------------- src/Common/HashTable/HashTableKeyHolder.h | 2 +- 2 files changed, 1 insertion(+), 39 deletions(-) diff --git a/src/Common/HashTable/ClearableHashSet.h b/src/Common/HashTable/ClearableHashSet.h index 006d45df7cd..13067c39d1b 100644 --- a/src/Common/HashTable/ClearableHashSet.h +++ b/src/Common/HashTable/ClearableHashSet.h @@ -48,30 +48,6 @@ struct ClearableHashTableCell : public BaseCell ClearableHashTableCell(const Key & key_, const State & state) : BaseCell(key_, state), version(state.version) {} }; -using StringRefBaseCell = HashSetCellWithSavedHash, ClearableHashSetState>; - -/// specialization for StringRef to allow zero size key (empty string) -template <> -struct ClearableHashTableCell : public StringRefBaseCell -{ - using State = ClearableHashSetState; - using value_type = typename StringRefBaseCell::value_type; - - UInt32 version; - - bool isZero(const State & state) const { return version != state.version; } - static bool isZero(const StringRef & key_, const State & state_) { return StringRefBaseCell::isZero(key_, state_); } - - /// Set the key value to zero. - void setZero() { version = 0; } - - /// Do I need to store the zero key separately (that is, can a zero key be inserted into the hash table). - static constexpr bool need_zero_value_storage = true; - - ClearableHashTableCell() { } /// NOLINT - ClearableHashTableCell(const StringRef & key_, const State & state) : StringRefBaseCell(key_, state), version(state.version) { } -}; - template < typename Key, typename Hash = DefaultHash, @@ -90,13 +66,6 @@ public: { ++this->version; this->m_size = 0; - - if constexpr (Cell::need_zero_value_storage) - { - /// clear ZeroValueStorage - if (this->hasZero()) - this->clearHasZero(); - } } }; @@ -119,13 +88,6 @@ public: { ++this->version; this->m_size = 0; - - if constexpr (Cell::need_zero_value_storage) - { - /// clear ZeroValueStorage - if (this->hasZero()) - this->clearHasZero(); - } } }; diff --git a/src/Common/HashTable/HashTableKeyHolder.h b/src/Common/HashTable/HashTableKeyHolder.h index d10ad093af6..994587e67ec 100644 --- a/src/Common/HashTable/HashTableKeyHolder.h +++ b/src/Common/HashTable/HashTableKeyHolder.h @@ -89,7 +89,7 @@ inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::ArenaKeyHolder & holder) inline void ALWAYS_INLINE keyHolderPersistKey(DB::ArenaKeyHolder & holder) { // Hash table shouldn't ask us to persist a zero key - assert(holder.key.size > 0); + // assert(holder.key.size > 0); holder.key.data = holder.pool.insert(holder.key.data, holder.key.size); } From 51c69b6fbdefa577a5e8ee10819fc5ec91d6aa9b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 May 2023 17:36:35 +0000 Subject: [PATCH 018/129] Comments regarding zero values in clearable hash table --- src/Common/HashTable/ClearableHashSet.h | 4 ++++ src/Common/HashTable/HashTableKeyHolder.h | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Common/HashTable/ClearableHashSet.h b/src/Common/HashTable/ClearableHashSet.h index 13067c39d1b..657d388cbfb 100644 --- a/src/Common/HashTable/ClearableHashSet.h +++ b/src/Common/HashTable/ClearableHashSet.h @@ -10,6 +10,10 @@ * Instead of this class, you could just use the pair (version, key) in the HashSet as the key * but then the table would accumulate all the keys that it ever stored, and it was unreasonably growing. * This class goes a step further and considers the keys with the old version empty in the hash table. + * + * Zero values note: + * A cell in ClearableHashSet can store a zero values as normal value + * If its version is equal to the version of the set itself, then it's not considered as empty even key's value is zero value of the corresponding type */ diff --git a/src/Common/HashTable/HashTableKeyHolder.h b/src/Common/HashTable/HashTableKeyHolder.h index 994587e67ec..d1b38744255 100644 --- a/src/Common/HashTable/HashTableKeyHolder.h +++ b/src/Common/HashTable/HashTableKeyHolder.h @@ -88,7 +88,11 @@ inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::ArenaKeyHolder & holder) inline void ALWAYS_INLINE keyHolderPersistKey(DB::ArenaKeyHolder & holder) { - // Hash table shouldn't ask us to persist a zero key + // Normally, our hash table shouldn't ask us to persist a zero key, + // but it can happened in the case of clearable hash table (ClearableHashSet, for example). + // The clearable hash table doesn't use zero storage and + // distinguishes empty keys by using cell version, not the value itself. + // So, when an empty StringRef is inserted in ClearableHashSet we'll get here key of zero size. // assert(holder.key.size > 0); holder.key.data = holder.pool.insert(holder.key.data, holder.key.size); } From c87b33a24dde639144648ac9c648844ef4dc1c07 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Sat, 13 May 2023 22:55:27 -0700 Subject: [PATCH 019/129] Fix error on OS X regarding resetting timeouts. This happens when remote disconnects due to inactivity. It seems to work on Linux, likely due to difference in SO_LINGER, maybe a different default timeout on Darwin. Verified manually using clickhouse cloud using following process: 1. Connect to instance. 2. Run `show tables`. 3. Wait 6 minutes. 4. Run `show tables`. With this fix, the EINVAL is not reported, and client will simply reconnect. --- src/IO/TimeoutSetter.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/IO/TimeoutSetter.cpp b/src/IO/TimeoutSetter.cpp index 1beb26772e5..ed21383ccd4 100644 --- a/src/IO/TimeoutSetter.cpp +++ b/src/IO/TimeoutSetter.cpp @@ -31,13 +31,15 @@ TimeoutSetter::~TimeoutSetter() { try { + bool connected = socket.impl()->initialized(); + if (!connected) + return; + socket.setSendTimeout(old_send_timeout); socket.setReceiveTimeout(old_receive_timeout); } catch (...) { - /// Sometimes caught on Mac OS X. This message can be safely ignored. - /// If you are developer using Mac, please debug this error message by yourself. tryLogCurrentException("Client", "TimeoutSetter: Can't reset timeouts"); } } From b06e34a77fb597b53637ec9b37e0bbebbfd98a1f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 10 May 2023 20:54:59 -0300 Subject: [PATCH 020/129] Accept key value delimiter as part of value --- .../keyvaluepair/impl/NeedleFactory.h | 22 ++++++++++++++++++- .../keyvaluepair/impl/StateHandlerImpl.h | 18 +++++---------- ...t_key_value_pairs_multiple_input.reference | 16 ++++++++++++-- ...extract_key_value_pairs_multiple_input.sql | 14 +++++++++++- 4 files changed, 54 insertions(+), 16 deletions(-) diff --git a/src/Functions/keyvaluepair/impl/NeedleFactory.h b/src/Functions/keyvaluepair/impl/NeedleFactory.h index 007f970aedf..b5f0645abcc 100644 --- a/src/Functions/keyvaluepair/impl/NeedleFactory.h +++ b/src/Functions/keyvaluepair/impl/NeedleFactory.h @@ -38,7 +38,7 @@ public: return SearchSymbols {std::string{needles.data(), needles.size()}}; } - SearchSymbols getReadNeedles(const Configuration & extractor_configuration) + SearchSymbols getReadKeyNeedles(const Configuration & extractor_configuration) { const auto & [key_value_delimiter, quoting_character, pair_delimiters] = extractor_configuration; @@ -57,6 +57,26 @@ public: return SearchSymbols {std::string{needles.data(), needles.size()}}; } + + SearchSymbols getReadValueNeedles(const Configuration & extractor_configuration) + { + const auto & [key_value_delimiter, quoting_character, pair_delimiters] + = extractor_configuration; + + std::vector needles; + + needles.push_back(quoting_character); + + std::copy(pair_delimiters.begin(), pair_delimiters.end(), std::back_inserter(needles)); + + if constexpr (WITH_ESCAPING) + { + needles.push_back('\\'); + } + + return SearchSymbols {std::string{needles.data(), needles.size()}}; + } + SearchSymbols getReadQuotedNeedles(const Configuration & extractor_configuration) { const auto quoting_character = extractor_configuration.quoting_character; diff --git a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h index 217a42b84ad..7fc3ba54833 100644 --- a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h +++ b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h @@ -41,7 +41,8 @@ public: NeedleFactory needle_factory; wait_needles = needle_factory.getWaitNeedles(configuration); - read_needles = needle_factory.getReadNeedles(configuration); + read_key_needles = needle_factory.getReadKeyNeedles(configuration); + read_value_needles = needle_factory.getReadValueNeedles(configuration); read_quoted_needles = needle_factory.getReadQuotedNeedles(configuration); } @@ -77,7 +78,7 @@ public: size_t pos = 0; - while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_needles)) + while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_key_needles)) { auto character_position = p - file.begin(); size_t next_pos = character_position + 1u; @@ -191,10 +192,6 @@ public: { return {pos + 1u, State::READING_QUOTED_VALUE}; } - else if (isKeyValueDelimiter(current_character)) - { - return {pos, State::WAITING_KEY}; - } if constexpr (WITH_ESCAPING) { @@ -218,7 +215,7 @@ public: size_t pos = 0; - while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_needles)) + while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_value_needles)) { const size_t character_position = p - file.begin(); size_t next_pos = character_position + 1u; @@ -237,10 +234,6 @@ public: } } } - else if (isKeyValueDelimiter(*p)) - { - return {next_pos, State::WAITING_KEY}; - } else if (isPairDelimiter(*p)) { value.append(file.begin() + pos, file.begin() + character_position); @@ -300,7 +293,8 @@ public: private: SearchSymbols wait_needles; - SearchSymbols read_needles; + SearchSymbols read_key_needles; + SearchSymbols read_value_needles; SearchSymbols read_quoted_needles; /* diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference index 125afa19427..e3bd1d7fc45 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference @@ -123,7 +123,7 @@ SELECT -- semi-colon as pair delimiter -- expected output: {'age':'31','name':'neymar','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;invalid1:invalid1,invalid2:invalid2', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) @@ -132,7 +132,7 @@ WITH ) AS x SELECT x; -{'age':'31','name':'neymar','team':'psg'} +{'age':'31','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} -- both comma and semi-colon as pair delimiters -- expected output: {'age':'31','last_key':'last_value','name':'neymar','nationality':'brazil','team':'psg'} WITH @@ -304,6 +304,18 @@ WITH SELECT x; {} +-- key value delimiter should be considered valid part of value, char2 has a problem with == +WITH + extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, + CAST( + arrayMap( + (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) + ), + 'Map(String,String)' + ) AS x +SELECT + x; +{'argument1':'1','argument2':'2','char':'=','char2':'=','formula':'1+2=3','result':'3','string':'foo=bar'} -- check str_to_map alias (it is case-insensitive) WITH sTr_tO_mAp('name:neymar, age:31 team:psg,nationality:brazil') AS s_map, diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql index 0a1500b1796..578a30560c8 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql @@ -123,7 +123,7 @@ SELECT -- semi-colon as pair delimiter -- expected output: {'age':'31','name':'neymar','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;invalid1:invalid1,invalid2:invalid2', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) @@ -427,6 +427,18 @@ WITH SELECT x; +-- key value delimiter should be considered valid part of value, char2 has a problem with == +WITH + extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, + CAST( + arrayMap( + (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) + ), + 'Map(String,String)' + ) AS x +SELECT + x; + -- check str_to_map alias (it is case-insensitive) WITH sTr_tO_mAp('name:neymar, age:31 team:psg,nationality:brazil') AS s_map, From fff0cf28b70054ca2ceb1e1f0bedd57fdf90a4f2 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 11 May 2023 09:30:09 -0300 Subject: [PATCH 021/129] address pr comments --- .../02499_extract_key_value_pairs_multiple_input.reference | 6 +++--- .../02499_extract_key_value_pairs_multiple_input.sql | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference index e3bd1d7fc45..ccccec10667 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference @@ -121,9 +121,9 @@ SELECT x; {} -- semi-colon as pair delimiter --- expected output: {'age':'31','name':'neymar','team':'psg'} +-- expected output: {'age':'31','anotherkey':'anothervalue','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value;anotherkey:anothervalue', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) @@ -132,7 +132,7 @@ WITH ) AS x SELECT x; -{'age':'31','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} +{'age':'31','anotherkey':'anothervalue','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} -- both comma and semi-colon as pair delimiters -- expected output: {'age':'31','last_key':'last_value','name':'neymar','nationality':'brazil','team':'psg'} WITH diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql index 578a30560c8..58975bdfeec 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql @@ -121,9 +121,9 @@ SELECT x; -- semi-colon as pair delimiter --- expected output: {'age':'31','name':'neymar','team':'psg'} +-- expected output: {'age':'31','anotherkey':'anothervalue','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value;anotherkey:anothervalue', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) From c16cba48b1d3895aa29114309679c15aed9ca60e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 11 May 2023 09:32:26 -0300 Subject: [PATCH 022/129] remove old / debug comment in tests --- .../02499_extract_key_value_pairs_multiple_input.reference | 2 +- .../02499_extract_key_value_pairs_multiple_input.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference index ccccec10667..a56e0881471 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference @@ -304,7 +304,7 @@ WITH SELECT x; {} --- key value delimiter should be considered valid part of value, char2 has a problem with == +-- key value delimiter should be considered valid part of value WITH extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, CAST( diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql index 58975bdfeec..5a1727f91e2 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql @@ -427,7 +427,7 @@ WITH SELECT x; --- key value delimiter should be considered valid part of value, char2 has a problem with == +-- key value delimiter should be considered valid part of value WITH extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, CAST( From 05ae7b2c2d7337c2ade92634f859718061a31064 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 May 2023 18:28:12 +0200 Subject: [PATCH 023/129] fix some tests --- src/Interpreters/MutationsInterpreter.cpp | 1 + src/Interpreters/MutationsInterpreter.h | 4 +-- .../MergeTree/MutateFromLogEntryTask.cpp | 5 ++- src/Storages/MergeTree/MutateTask.cpp | 30 +++++++++++++++-- .../ReplicatedMergeTreeMutationEntry.cpp | 9 +++++ .../ReplicatedMergeTreeMutationEntry.h | 2 ++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 11 ++++--- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++-- tests/clickhouse-test | 8 ++--- .../test_default_compression_codec/test.py | 4 +++ .../02440_mutations_finalization.reference | 5 +++ .../02440_mutations_finalization.sql | 33 +++++++++++++++++++ ...441_alter_delete_and_drop_column.reference | 2 ++ .../02441_alter_delete_and_drop_column.sql | 20 +++++++++++ 15 files changed, 130 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02440_mutations_finalization.reference create mode 100644 tests/queries/0_stateless/02440_mutations_finalization.sql create mode 100644 tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference create mode 100644 tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2f05ba5d7f8..2f9e647b99e 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -542,6 +542,7 @@ void MutationsInterpreter::prepare(bool dry_run) if (commands.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty mutation commands list"); + /// TODO Should we get columns, indices and projections from the part itself? Table metadata may be different const ColumnsDescription & columns_desc = metadata_snapshot->getColumns(); const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices(); const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections(); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 4b965546408..8bb90365050 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -50,7 +50,7 @@ public: bool return_all_columns_ = false, bool return_mutated_rows_ = false); - /// Special case for MergeTree + /// Special case for *MergeTree MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, @@ -123,7 +123,7 @@ public: private: StoragePtr storage; - /// Special case for MergeTree. + /// Special case for *MergeTree. MergeTreeData * data = nullptr; MergeTreeData::DataPartPtr part; }; diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 35a503c658d..d0b85ee65b8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -90,7 +90,10 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() } new_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, storage.format_version); - commands = std::make_shared(storage.queue.getMutationCommands(source_part, new_part_info.mutation)); + Strings mutation_ids; + commands = std::make_shared(storage.queue.getMutationCommands(source_part, new_part_info.mutation, mutation_ids)); + LOG_TRACE(log, "Mutating part {} with mutation commands from {} mutations ({}): {}", + entry.new_part_name, commands->size(), fmt::join(mutation_ids, ", "), commands->toString()); /// Once we mutate part, we must reserve space on the same disk, because mutations can possibly create hardlinks. /// Can throw an exception. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a3fa210ac42..27e5319ed4f 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -58,7 +58,9 @@ static void splitAndModifyMutationCommands( MergeTreeData::DataPartPtr part, const MutationCommands & commands, MutationCommands & for_interpreter, - MutationCommands & for_file_renames) + MutationCommands & for_file_renames, + const StorageMetadataPtr & table_metadata_snapshot, + Poco::Logger * log) { auto part_columns = part->getColumnsDescription(); @@ -142,6 +144,29 @@ static void splitAndModifyMutationCommands( { if (!mutated_columns.contains(column.name)) { + if (!table_metadata_snapshot->getColumns().has(column.name)) + { + /// We cannot add the column because there's no such column in table. + /// It's okay if the column was dropped. It may also absent in dropped_columns + /// if the corresponding MUTATE_PART entry was not created yet or was created separately from current MUTATE_PART. + /// But we don't know for sure what happened. + auto part_metadata_version = part->getMetadataVersion(); + auto table_metadata_version = table_metadata_snapshot->getMetadataVersion(); + if (table_metadata_version <= part_metadata_version) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent " + "in table {} with metadata version {}", + part->name, part_metadata_version, column.name, + part->storage.getStorageID().getNameForLogs(), table_metadata_version); + + if (part_metadata_version < table_metadata_version) + { + LOG_WARNING(log, "Ignoring column {} from part {} with metadata version {} because there is no such column " + "in table {} with metadata version {}. Assuming the column was dropped", column.name, part->name, + part_metadata_version, part->storage.getStorageID().getNameForLogs(), table_metadata_version); + continue; + } + } + for_interpreter.emplace_back( MutationCommand{.type = MutationCommand::Type::READ_COLUMN, .column_name = column.name, .data_type = column.type}); } @@ -1776,7 +1801,8 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); - MutationHelpers::splitAndModifyMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames); + MutationHelpers::splitAndModifyMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, + ctx->for_file_renames, ctx->metadata_snapshot, ctx->log); ctx->stage_progress = std::make_unique(1.0); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 1efb3f6826b..5e01cd96f6b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -99,4 +99,13 @@ std::shared_ptr ReplicatedMergeTreeMutationEntry::backup() c return std::make_shared(out.str()); } + +String ReplicatedMergeTreeMutationEntry::getBlockNumbersForLogs() const +{ + WriteBufferFromOwnString out; + for (const auto & kv : block_numbers) + out << kv.first << " = " << kv.second << "; "; + return out.str(); +} + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index 8c359a57279..09787bd1731 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -51,6 +51,8 @@ struct ReplicatedMergeTreeMutationEntry bool isAlterMutation() const { return alter_version != -1; } std::shared_ptr backup() const; + + String getBlockNumbersForLogs() const; }; using ReplicatedMergeTreeMutationEntryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 1762c7aabe9..9f7ae3222a4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -955,13 +955,14 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C const String & partition_id = pair.first; Int64 block_num = pair.second; mutations_by_partition[partition_id].emplace(block_num, &mutation); - LOG_TRACE(log, "Adding mutation {} for partition {} for all block numbers less than {}", entry->znode_name, partition_id, block_num); } + LOG_TRACE(log, "Adding mutation {} for {} partitions (data versions: {})", + entry->znode_name, entry->block_numbers.size(), entry->getBlockNumbersForLogs()); /// Initialize `mutation.parts_to_do`. We cannot use only current_parts + virtual_parts here so we /// traverse all the queue and build correct state of parts_to_do. auto queue_representation = getQueueRepresentation(queue, format_version); - mutation.parts_to_do = getPartNamesToMutate(*entry, virtual_parts, queue_representation, format_version); + mutation.parts_to_do = getPartNamesToMutate(*entry, current_parts, queue_representation, format_version); if (mutation.parts_to_do.size() == 0) some_mutations_are_probably_done = true; @@ -1801,7 +1802,7 @@ std::map ReplicatedMergeTreeQueue::getAlterMutationCo } MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( - const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const + const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const { /// NOTE: If the corresponding mutation is not found, the error is logged (and not thrown as an exception) /// to allow recovering from a mutation that cannot be executed. This way you can delete the mutation entry @@ -1840,6 +1841,8 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( MutationCommands commands; for (auto it = begin; it != end; ++it) { + chassert(mutation_pointer < it->second->entry->znode_name); + mutation_ids.push_back(it->second->entry->znode_name); const auto & commands_from_entry = it->second->entry->commands; commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); } @@ -2600,7 +2603,7 @@ void ReplicatedMergeTreeQueue::removeCurrentPartsFromMutations() { std::lock_guard state_lock(state_mutex); for (const auto & part_name : current_parts.getParts()) - removeCoveredPartsFromMutations(part_name, /*remove_part = */ true, /*remove_covered_parts = */ true); + removeCoveredPartsFromMutations(part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 368f2d4bc1f..3fefc341bbc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -392,7 +392,8 @@ public: /// Returns functor which used by MergeTreeMergerMutator to select parts for merge ReplicatedMergeTreeMergePredicate getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, PartitionIdsHint && partition_ids_hint); - MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const; + MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, + Strings & mutation_ids) const; /// Return mutation commands for part which could be not applied to /// it according to part mutation version. Used when we apply alter commands on fly, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b5e53950a02..5ca97f06cde 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5265,12 +5265,12 @@ void StorageReplicatedMergeTree::alter( fs::path(zookeeper_path) / "log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential)); PartitionBlockNumbersHolder partition_block_numbers_holder; + ReplicatedMergeTreeMutationEntry mutation_entry; if (have_mutation) { delayMutationOrThrowIfNeeded(&partial_shutdown_event, query_context); const String mutations_path(fs::path(zookeeper_path) / "mutations"); - ReplicatedMergeTreeMutationEntry mutation_entry; mutation_entry.alter_version = new_metadata_version; mutation_entry.source_replica = replica_name; mutation_entry.commands = std::move(maybe_mutation_commands); @@ -5322,12 +5322,16 @@ void StorageReplicatedMergeTree::alter( /// ReplicatedMergeTreeMutationEntry record in /mutations String mutation_path = dynamic_cast(*results[mutation_path_idx]).path_created; mutation_znode = mutation_path.substr(mutation_path.find_last_of('/') + 1); + LOG_DEBUG(log, "Created log entry {} to update table metadata to version {}, created a mutation {} (data versions: {})", + alter_entry->znode_name, alter_entry->alter_version, *mutation_znode, mutation_entry.getBlockNumbersForLogs()); } else { /// ALTER_METADATA record in replication /log String alter_path = dynamic_cast(*results[alter_path_idx]).path_created; alter_entry->znode_name = alter_path.substr(alter_path.find_last_of('/') + 1); + LOG_DEBUG(log, "Created log entry {} to update table metadata to version {}", + alter_entry->znode_name, alter_entry->alter_version); } break; } @@ -6493,7 +6497,8 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte const String & path_created = dynamic_cast(responses[1].get())->path_created; mutation_entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1); - LOG_TRACE(log, "Created mutation with ID {}", mutation_entry.znode_name); + LOG_TRACE(log, "Created mutation with ID {} (data versions: {})", + mutation_entry.znode_name, mutation_entry.getBlockNumbersForLogs()); break; } else if (rc == Coordination::Error::ZBADVERSION) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e279b899a93..cd9dde28fea 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -286,11 +286,11 @@ def get_processlist_with_stacktraces(args): -- NOTE: view() here to do JOIN on shards, instead of initiator FROM clusterAllReplicas('test_cluster_database_replicated', view( SELECT + p.*, arrayStringConcat(groupArray('Thread ID ' || toString(s.thread_id) || '\n' || arrayStringConcat(arrayMap( x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))), s.trace), '\n') AS stacktrace - )) AS stacktraces, - p.* + )) AS stacktraces FROM system.processes p JOIN system.stack_trace s USING (query_id) WHERE query NOT LIKE '%system.processes%' @@ -307,11 +307,11 @@ def get_processlist_with_stacktraces(args): args, """ SELECT + p.*, arrayStringConcat(groupArray('Thread ID ' || toString(s.thread_id) || '\n' || arrayStringConcat(arrayMap( x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))), s.trace), '\n') AS stacktrace - )) AS stacktraces, - p.* + )) AS stacktraces FROM system.processes p JOIN system.stack_trace s USING (query_id) WHERE query NOT LIKE '%system.processes%' diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index 5d033ac8f7e..c7c30f5eea4 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -262,6 +262,8 @@ def test_default_codec_multiple(start_cluster): ) ) + node2.query("SYSTEM SYNC REPLICA compression_table_multiple", timeout=15) + # Same codec for all assert ( get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_0") @@ -330,6 +332,8 @@ def test_default_codec_multiple(start_cluster): node1.query("OPTIMIZE TABLE compression_table_multiple FINAL") + node2.query("SYSTEM SYNC REPLICA compression_table_multiple", timeout=15) + assert ( get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING["Multiple"] diff --git a/tests/queries/0_stateless/02440_mutations_finalization.reference b/tests/queries/0_stateless/02440_mutations_finalization.reference new file mode 100644 index 00000000000..a8b9c2acdce --- /dev/null +++ b/tests/queries/0_stateless/02440_mutations_finalization.reference @@ -0,0 +1,5 @@ +0000000000 UPDATE n = 2 WHERE n = 1 ['all_0_0_0'] 0 +1 +0000000000 UPDATE n = 2 WHERE n = 1 ['all_0_0_0'] 0 +2 +0000000000 UPDATE n = 2 WHERE n = 1 [] 1 diff --git a/tests/queries/0_stateless/02440_mutations_finalization.sql b/tests/queries/0_stateless/02440_mutations_finalization.sql new file mode 100644 index 00000000000..796dcde8e4e --- /dev/null +++ b/tests/queries/0_stateless/02440_mutations_finalization.sql @@ -0,0 +1,33 @@ + +create table mut (n int) engine=ReplicatedMergeTree('/test/02440/{database}/mut', '1') order by tuple(); +set insert_keeper_fault_injection_probability=0; +insert into mut values (1); +system stop merges mut; +alter table mut update n = 2 where n = 1; +-- it will create MUTATE_PART entry, but will not execute it + +select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; + +-- merges (and mutations) will start again after detach/attach, we need to avoid this somehow... +create table tmp (n int) engine=MergeTree order by tuple() settings index_granularity=1; +insert into tmp select * from numbers(1000); +alter table tmp update n = sleepEachRow(1) where 1; +select sleepEachRow(2) as higher_probablility_of_reproducing_the_issue format Null; + +-- it will not execute MUTATE_PART, because another mutation is currently executing (in tmp) +alter table mut modify setting max_number_of_mutations_for_replica=1; +detach table mut; +attach table mut; + +-- mutation should not be finished yet +select * from mut; +select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; + +alter table mut modify setting max_number_of_mutations_for_replica=100; +system sync replica mut; + +-- and now it should +select * from mut; +select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; + +drop table tmp; -- btw, it will check that mutation can be cancelled between blocks on shutdown diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference new file mode 100644 index 00000000000..e9858167301 --- /dev/null +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference @@ -0,0 +1,2 @@ +MUTATE_PART all_0_0_0_1 ['all_0_0_0'] +1 2 diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql new file mode 100644 index 00000000000..d274fae1a4f --- /dev/null +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql @@ -0,0 +1,20 @@ + +create table mut (n int, m int, k int) engine=ReplicatedMergeTree('/test/02441/{database}/mut', '1') order by n; +set insert_keeper_fault_injection_probability=0; +insert into mut values (1, 2, 3), (10, 20, 30); + +system stop merges mut; +alter table mut delete where n = 10; +alter table mut drop column k settings alter_sync=0; +system sync replica mut pull; + +-- a funny way to wait for ALTER_METADATA to disappear from the replication queue +select sleepEachRow(1) from url('http://localhost:8123/?param_tries={1..30}&query=' || encodeURLComponent( + 'select * from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'' and type=''ALTER_METADATA''' + ), 'LineAsString', 's String') settings max_threads=1 format Null; + +select type, new_part_name, parts_to_merge from system.replication_queue where database=currentDatabase() and table='mut'; +system start merges mut; +set receive_timeout=30; +system sync replica mut; +select * from mut; From 65bc702b0bfa2dc01c76fc1ba10007eff980fdd7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 May 2023 20:02:30 +0200 Subject: [PATCH 024/129] fix --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 27e5319ed4f..f7de6ed3d22 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -144,7 +144,7 @@ static void splitAndModifyMutationCommands( { if (!mutated_columns.contains(column.name)) { - if (!table_metadata_snapshot->getColumns().has(column.name)) + if (!table_metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtuals().contains(column.name)) { /// We cannot add the column because there's no such column in table. /// It's okay if the column was dropped. It may also absent in dropped_columns From 29aa96037758df616ec6253289e218abd7212626 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 09:07:35 +0200 Subject: [PATCH 025/129] refine docs for regexp tree dictionary --- docs/en/sql-reference/dictionaries/index.md | 107 ++++++++++++++++---- 1 file changed, 89 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 5801b7866cb..ad5a65df994 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2197,13 +2197,13 @@ Result: └─────────────────────────────────┴───────┘ ``` -## RegExp Tree Dictionary {#regexp-tree-dictionary} +## RegExpTree Dictionary {#regexp-tree-dictionary} -Regexp Tree dictionary stores multiple trees of regular expressions with attributions. Users can retrieve strings in the dictionary. If a string matches the root of the regexp tree, we will collect the corresponding attributes of the matched root and continue to walk the children. If any of the children matches the string, we will collect attributes and rewrite the old ones if conflicts occur, then continue the traverse until we reach leaf nodes. +RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some senarioes, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environment. -Example of the ddl query for creating Regexp Tree dictionary: +### Use RegExpTree Dictionary in local environment - +In local environment, we create RegexpTree dictionary by a yaml file: ```sql create dictionary regexp_dict @@ -2218,11 +2218,9 @@ LAYOUT(regexp_tree) ... ``` -**Source** +The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree. For example: -We introduce a type of source called `YAMLRegExpTree` representing the structure of Regexp Tree dictionary. An Example of a valid yaml config is like: - -```xml +```yaml - regexp: 'Linux/(\d+[\.\d]*).+tlinux' name: 'TencentOS' version: '\1' @@ -2240,17 +2238,15 @@ We introduce a type of source called `YAMLRegExpTree` representing the structure version: '10' ``` -The key `regexp` represents the regular expression of a tree node. The name of key is same as the dictionary key. The `name` and `version` is user-defined attributions in the dicitionary. The `versions` (which can be any name that not appear in attributions or the key) indicates the children nodes of this tree. +This config consists of a list of RegExpTree nodes. Each node has following structure: -**Back Reference** +- **regexp** means the regular expression of this node. +- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have the both attributions. The second node only have `name` attribution, because the `version` is defined in the children nodes. + - The value of an attribution could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. +- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, chilren nodes have more priority. + - the name of **children nodes** in yaml files can be arbitrary. -The value of an attribution could contain a back reference which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. - -During the query execution, the back reference in the value will be replaced by the matched capture group. - -**Query** - -Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull` work with it. +Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. Example: @@ -2260,12 +2256,87 @@ SELECT dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024'); Result: -``` +```text ┌─dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024')─┐ │ ('Andriod','12') │ └─────────────────────────────────────────────────────────────────┘ ``` +Explain: + +In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the first layer, so the dictionary will continue to look into the children nodes in the second layer and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Andriod` defined in the first layer and the value of `version` is `12` defined in the second layer. + +### Use RegExpTree Dictionary on cloud + +We have shown how RegExpTree work in the local enviroument, but we cannot use `YAMLRegExpTree` on cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local enviroment, then dump this dictionary to a csv file by `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. + +```sql +select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') +``` + +The content of csv file is: + +```text +1,0,"Linux/(\d+[\.\d]*).+tlinux","['version','name']","['\\1','TencentOS']" +2,0,"(\d+)/tclwebkit(\d+[\.\d]*)","['comment','version','name']","['test $1 and $2','$1','Andriod']" +3,2,"33/tclwebkit","['version']","['13']" +4,2,"3[12]/tclwebkit","['version']","['12']" +5,2,"3[12]/tclwebkit","['version']","['11']" +6,2,"3[12]/tclwebkit","['version']","['10']" +``` + +The schema of dumped file is always + +- `id UInt64` represents the identify number of the RegexpTree node. +- `parent_id UInt64` represents the id of the parent of a node. +- `regexp String` represents the regular expression string. +- `keys Array(String)` represents the names of user defined attributions. +- `values Array(String)` represents the values of user defined attributions. + +On the cloud, we can create a table `regexp_dictionary_source_table` with the above table structure. + +```sql +CREATE TABLE regexp_dictionary_source_table +( + id UInt64, + parent_id UInt64, + regexp String, + keys Array(String), + values Array(String) +) ENGINE=Memory; +``` + +Then update the local CSV by + +```bash +clickhouse client \ + --host MY_HOST \ + --secure \ + --password MY_PASSWORD \ + --query " + insert into regexp_dictionary_source_table + select * from input ('id UInt64, parent_id UInt64, regexp String, keys Array(String), values Array(String)') + FORMAT CSV" < regexp_dict.csv +``` + +You can see how to [Insert Local Files](https://clickhouse.com/docs/en/integrations/data-ingestion/insert-local-files) for more details. After we initialize the source table, we can create a RegexpTree by table source: + +``` sql +create dictionary regexp_dict +( + regexp String, + name String, + version String +PRIMARY KEY(regexp) +SOURCE(CLICKHOUSE(TABLE 'regexp_dictionary_source_table')) +LIFETIME(0) +LAYOUT(regexp_tree); +``` + +### Use RegexpTree Dictionary as a UA Parser + +With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demostrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) + ## Embedded Dictionaries {#embedded-dictionaries} From e4e473ef30ff4a6ca3f198e3da6d21b1ff85ccbd Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:14 +0200 Subject: [PATCH 026/129] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index ad5a65df994..1ab6370c977 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2199,7 +2199,7 @@ Result: ## RegExpTree Dictionary {#regexp-tree-dictionary} -RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some senarioes, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environment. +RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some scenarios, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environments. ### Use RegExpTree Dictionary in local environment From 31b8e3c4892ff03340314274ab715506aff0e95b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:24 +0200 Subject: [PATCH 027/129] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 1ab6370c977..b50ad4bf365 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2241,9 +2241,9 @@ The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree This config consists of a list of RegExpTree nodes. Each node has following structure: - **regexp** means the regular expression of this node. -- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have the both attributions. The second node only have `name` attribution, because the `version` is defined in the children nodes. +- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have both attributions. The second node only has `name` attribution, because the `version` is defined in the children nodes. - The value of an attribution could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. -- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, chilren nodes have more priority. +- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, children nodes have more priority. - the name of **children nodes** in yaml files can be arbitrary. Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. From ed5906f15ded79823b9b4988cca6b335a6908100 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:31 +0200 Subject: [PATCH 028/129] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index b50ad4bf365..086c0b5c0ed 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2335,7 +2335,7 @@ LAYOUT(regexp_tree); ### Use RegexpTree Dictionary as a UA Parser -With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demostrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) +With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) ## Embedded Dictionaries {#embedded-dictionaries} From a40d86b921ac3a519b7c9d65b41afbdf6667d2b8 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:42 +0200 Subject: [PATCH 029/129] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 086c0b5c0ed..66f661cce60 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2268,7 +2268,7 @@ In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in t ### Use RegExpTree Dictionary on cloud -We have shown how RegExpTree work in the local enviroument, but we cannot use `YAMLRegExpTree` on cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local enviroment, then dump this dictionary to a csv file by `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. +We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. ```sql select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') From 68138395ebb4b7c129f9ef8c6c8fbd2a8dfc1c9b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 5 May 2023 15:31:12 +0200 Subject: [PATCH 030/129] Fix parameterized views when query parameter used multiple times in the query Example: CREATE VIEW view AS SELECT * FROM system.one WHERE dummy = {k1:Int}+1 OR dummy = {k1:Int}+2 ^^ ^^ Signed-off-by: Azat Khuzhin --- src/Interpreters/ActionsVisitor.cpp | 15 +++++++++++++-- .../02428_parameterized_view.reference | 3 ++- .../0_stateless/02428_parameterized_view.sh | 5 +++-- 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b3bb6e9db90..28ba2e3e838 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1216,11 +1216,22 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & else if (data.is_create_parameterized_view && query_parameter) { const auto data_type = DataTypeFactory::instance().get(query_parameter->type); - ColumnWithTypeAndName column(data_type,query_parameter->getColumnName()); + /// Use getUniqueName() to allow multiple use of query parameter in the query: + /// + /// CREATE VIEW view AS + /// SELECT * + /// FROM system.one + /// WHERE dummy = {k1:Int}+1 OR dummy = {k1:Int}+2 + /// ^^ ^^ + /// + /// NOTE: query in the VIEW will not be modified this is needed + /// only during analysis for CREATE VIEW to avoid duplicated + /// column names. + ColumnWithTypeAndName column(data_type, data.getUniqueName("__" + query_parameter->getColumnName())); data.addColumn(column); argument_types.push_back(data_type); - argument_names.push_back(query_parameter->name); + argument_names.push_back(column.name); } else { diff --git a/tests/queries/0_stateless/02428_parameterized_view.reference b/tests/queries/0_stateless/02428_parameterized_view.reference index 59a3f4d67dd..b73c52c478f 100644 --- a/tests/queries/0_stateless/02428_parameterized_view.reference +++ b/tests/queries/0_stateless/02428_parameterized_view.reference @@ -19,7 +19,8 @@ ERROR 50 ERROR 10 -ERROR +10 +20 20 ERROR 30 diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index 57a43ce08a0..3abfbfc22fc 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -62,7 +62,8 @@ $CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv2(price=50)" 2>&1 | grep $CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv3 AS SELECT * FROM test_02428_Catalog WHERE Price={price:UInt64} AND Quantity=3" $CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv3(price=10)" -$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv4 AS SELECT * FROM test_02428_Catalog WHERE Price={price:UInt64} AND Quantity={price:UInt64}" 2>&1 | grep -Fq "DUPLICATE_COLUMN" && echo 'ERROR' || echo 'OK' +$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv4 AS SELECT * FROM test_02428_Catalog WHERE Price={price:UInt64} OR Price={price:UInt64}*2" +$CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv4(price=10) ORDER BY Price" $CLICKHOUSE_CLIENT -q "CREATE DATABASE ${CLICKHOUSE_TEST_UNIQUE_NAME}" $CLICKHOUSE_CLIENT -q "CREATE TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog (Name String, Price UInt64, Quantity UInt64) ENGINE = Memory" @@ -130,4 +131,4 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog" $CLICKHOUSE_CLIENT -q "DROP DATABASE ${CLICKHOUSE_TEST_UNIQUE_NAME}" $CLICKHOUSE_CLIENT -q "DROP VIEW 02428_trace_view" $CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces_trace_id_ts" -$CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces" From ea5976180969fafb7dc7db8b5be5ff2d9c2976f0 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 15:25:04 +0200 Subject: [PATCH 031/129] fix OptimizeRegularExpression --- src/Common/OptimizedRegularExpression.cpp | 33 +++++++++++++++-------- src/Common/tests/gtest_optimize_re.cpp | 32 ++++++++++++---------- 2 files changed, 40 insertions(+), 25 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 68f5b86877e..92b3ad32ecd 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -63,12 +63,13 @@ const char * analyzeImpl( bool is_first_call = begin == regexp.data(); int depth = 0; is_trivial = true; + bool is_prefix = true; required_substring.clear(); bool has_alternative_on_depth_0 = false; bool has_case_insensitive_flag = false; /// Substring with a position. - using Substring = std::pair; + using Substring = std::pair; using Substrings = std::vector; Substrings trivial_substrings(1); @@ -98,6 +99,9 @@ const char * analyzeImpl( auto finish_non_trivial_char = [&](bool create_new_substr = true) { + is_trivial = false; + if (create_new_substr) + is_prefix = false; if (depth != 0) return; @@ -106,6 +110,7 @@ const char * analyzeImpl( if (alter.suffix) { alter.literal += last_substring->first; + alter.suffix = false; } } @@ -126,16 +131,24 @@ const char * analyzeImpl( if (alter.prefix) { alter.literal = last_substring->first + alter.literal; + alter.prefix = is_prefix; } } if (group_required_string.prefix) + { last_substring->first += group_required_string.literal; + last_substring->second = is_prefix; + } else { finish_non_trivial_char(); last_substring->first = group_required_string.literal; + last_substring->second = false; } + + is_prefix = is_prefix && group_required_string.prefix && group_required_string.suffix; + /// if we can still append, no need to finish it. e.g. abc(de)fg should capture abcdefg if (!last_substring->first.empty() && !group_required_string.suffix) { @@ -185,7 +198,6 @@ const char * analyzeImpl( goto ordinary; default: /// all other escape sequences are not supported - is_trivial = false; finish_non_trivial_char(); break; } @@ -196,6 +208,7 @@ const char * analyzeImpl( case '|': is_trivial = false; + is_prefix = false; ++pos; if (depth == 0) { @@ -205,6 +218,7 @@ const char * analyzeImpl( break; case '(': + /// bracket does not break is_prefix. for example abc(d) has a prefix 'abcd' is_trivial = false; if (!in_square_braces) { @@ -258,7 +272,6 @@ const char * analyzeImpl( case '[': in_square_braces = true; ++depth; - is_trivial = false; finish_non_trivial_char(); ++pos; break; @@ -270,7 +283,6 @@ const char * analyzeImpl( --depth; if (depth == 0) in_square_braces = false; - is_trivial = false; finish_non_trivial_char(); ++pos; break; @@ -284,7 +296,6 @@ const char * analyzeImpl( break; case '^': case '$': case '.': case '+': - is_trivial = false; finish_non_trivial_char(); ++pos; break; @@ -296,7 +307,6 @@ const char * analyzeImpl( case '?': [[fallthrough]]; case '*': - is_trivial = false; if (depth == 0 && !last_substring->first.empty() && !in_square_braces) { last_substring->first.resize(last_substring->first.size() - 1); @@ -318,8 +328,9 @@ const char * analyzeImpl( default: if (depth == 0 && !in_curly_braces && !in_square_braces) { + /// record the first position of last string. if (last_substring->first.empty()) - last_substring->second = pos - begin; + last_substring->second = is_prefix; last_substring->first.push_back(*pos); } ++pos; @@ -328,10 +339,9 @@ const char * analyzeImpl( } finish: - finish_non_trivial_char(false); - if (!is_trivial) { + finish_non_trivial_char(false); /// we calculate required substring even though has_alternative_on_depth_0. /// we will clear the required substring after putting it to alternatives. if (!has_case_insensitive_flag) @@ -357,7 +367,7 @@ finish: if (max_length >= MIN_LENGTH_FOR_STRSTR || (!is_first_call && max_length > 0)) { required_substring.literal = candidate_it->first; - required_substring.prefix = candidate_it->second == 0; + required_substring.prefix = candidate_it->second; required_substring.suffix = candidate_it + 1 == trivial_substrings.end(); } } @@ -365,7 +375,8 @@ finish: else if (!trivial_substrings.empty()) { required_substring.literal = trivial_substrings.front().first; - required_substring.prefix = trivial_substrings.front().second == 0; + /// trivial string means the whole regex is a simple string literal, so the prefix and suffix should be true. + required_substring.prefix = true; required_substring.suffix = true; } diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index 556700f1fcc..3710666d336 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -4,37 +4,40 @@ TEST(OptimizeRE, analyze) { - auto test_f = [](const std::string & regexp, const std::string & answer, std::vector expect_alternatives = {}, bool trival_expected = false) + auto test_f = [](const std::string & regexp, const std::string & required, std::vector expect_alternatives = {}, bool trival_expected = false, bool prefix_expected = false) { - std::string required; + std::string answer; bool is_trivial; bool is_prefix; std::vector alternatives; - OptimizedRegularExpression::analyze(regexp, required, is_trivial, is_prefix, alternatives); + OptimizedRegularExpression::analyze(regexp, answer, is_trivial, is_prefix, alternatives); std::cerr << regexp << std::endl; EXPECT_EQ(required, answer); EXPECT_EQ(alternatives, expect_alternatives); EXPECT_EQ(is_trivial, trival_expected); + EXPECT_EQ(is_prefix, prefix_expected); }; - test_f("abc", "abc", {}, true); + test_f("abc", "abc", {}, true, true); test_f("c([^k]*)de", ""); - test_f("abc(de)fg", "abcdefg"); - test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}); - test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}); + test_f("abc(de)fg", "abcdefg", {}, false, true); + test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}, false, true); + test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}, false, true); test_f("abc|fgk|xyz", "", {"abc","fgk", "xyz"}); - test_f("(abc)", "abc"); + test_f("(abc)", "abc", {}, false, true); test_f("(abc|fgk)", "", {"abc","fgk"}); test_f("(abc|fgk)(e|f|zkh|)", "", {"abc","fgk"}); test_f("abc(abc|fg)xyzz", "xyzz", {"abcabcxyzz","abcfgxyzz"}); + test_f("((abc|fg)kkk*)xyzz", "xyzz", {"abckk", "fgkk"}); + test_f("abc(*(abc|fg)*)xyzz", "xyzz"); test_f("abc[k]xyzz", "xyzz"); test_f("(abc[k]xyzz)", "xyzz"); - test_f("abc((de)fg(hi))jk", "abcdefghijk"); - test_f("abc((?:de)fg(?:hi))jk", "abcdefghijk"); - test_f("abc((de)fghi+zzz)jk", "abcdefghi"); - test_f("abc((de)fg(hi))?jk", "abc"); - test_f("abc((de)fghi?zzz)jk", "abcdefgh"); + test_f("abc((de)fg(hi))jk", "abcdefghijk", {}, false, true); + test_f("abc((?:de)fg(?:hi))jk", "abcdefghijk", {}, false, true); + test_f("abc((de)fghi+zzz)jk", "abcdefghi", {}, false, true); + test_f("abc((de)fg(hi))?jk", "abc", {}, false, true); + test_f("abc((de)fghi?zzz)jk", "abcdefgh", {}, false, true); test_f("abc(*cd)jk", "cdjk"); - test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"abcdefg", "abcxyzfg", "abc{xx}fg"}); + test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"abcdefg", "abcxyzfg", "abc{xx}fg"}, false, true); test_f("abc(abc|fg)?xyzz", "xyzz"); test_f("abc(abc|fg){0,1}xyzz", "xyzz"); test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bc"}); @@ -43,4 +46,5 @@ TEST(OptimizeRE, analyze) test_f(R"([Bb]ai[Dd]u[Ss]pider(?:-[A-Za-z]{1,30})(?:-[A-Za-z]{1,30}|)|bingbot|\bYeti(?:-[a-z]{1,30}|)|Catchpoint(?: bot|)|[Cc]harlotte|Daumoa(?:-feedfetcher|)|(?:[a-zA-Z]{1,30}-|)Googlebot(?:-[a-zA-Z]{1,30}|))", "", {"pider-", "bingbot", "Yeti-", "Yeti", "Catchpoint bot", "Catchpoint", "harlotte", "Daumoa-feedfetcher", "Daumoa", "-Googlebot", "Googlebot"}); test_f("abc|(:?xx|yy|zz|x?)def", "", {"abc", "def"}); test_f("abc|(:?xx|yy|zz|x?){1,2}def", "", {"abc", "def"}); + test_f(R"(\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z)", "/k8s1"); } From 7df0e9d933267d7a2595e7d970c190650e09954c Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 15:33:08 +0200 Subject: [PATCH 032/129] fix broken link --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 66f661cce60..4abc41cdf42 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2268,7 +2268,7 @@ In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in t ### Use RegExpTree Dictionary on cloud -We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. +We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. ```sql select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') From dea5cbcf4e01af9793ba1beac777c88a7567cc8c Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Tue, 16 May 2023 16:39:00 +0200 Subject: [PATCH 033/129] Slightly update comment --- src/Common/HashTable/HashTableKeyHolder.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/HashTable/HashTableKeyHolder.h b/src/Common/HashTable/HashTableKeyHolder.h index d1b38744255..e2f9d80db53 100644 --- a/src/Common/HashTable/HashTableKeyHolder.h +++ b/src/Common/HashTable/HashTableKeyHolder.h @@ -88,7 +88,7 @@ inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::ArenaKeyHolder & holder) inline void ALWAYS_INLINE keyHolderPersistKey(DB::ArenaKeyHolder & holder) { - // Normally, our hash table shouldn't ask us to persist a zero key, + // Normally, our hash table shouldn't ask to persist a zero key, // but it can happened in the case of clearable hash table (ClearableHashSet, for example). // The clearable hash table doesn't use zero storage and // distinguishes empty keys by using cell version, not the value itself. From 77adb7c8bc4c680503d2937a8448f097fef1af3d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 16 May 2023 17:00:05 +0200 Subject: [PATCH 034/129] Update 02535_analyzer_group_by_use_nulls reference --- .../02535_analyzer_group_by_use_nulls.reference | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference index 50755627996..63610604ddd 100644 --- a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference @@ -5,15 +5,25 @@ GROUP BY ROLLUP(number, number % 2) ORDER BY (number, number % 2, val) SETTINGS group_by_use_nulls=1; 0 0 0 +0 \N 0 1 1 1 +1 \N 1 2 0 2 +2 \N 2 3 1 3 +3 \N 3 4 0 4 +4 \N 4 5 1 5 +5 \N 5 6 0 6 +6 \N 6 7 1 7 +7 \N 7 8 0 8 +8 \N 8 9 1 9 +9 \N 9 \N \N 45 set optimize_group_by_function_keys = 0; SELECT number, number % 2, sum(number) AS val From 0da82945ac2df6c79d7a47ae628346b3d1fda6e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 May 2023 18:18:48 +0200 Subject: [PATCH 035/129] fix --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 3 ++- .../0_stateless/02440_mutations_finalization.reference | 2 +- .../queries/0_stateless/02440_mutations_finalization.sql | 5 +++-- .../0_stateless/02441_alter_delete_and_drop_column.sql | 8 +++++++- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9f7ae3222a4..e3c9a54023c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1841,7 +1841,8 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( MutationCommands commands; for (auto it = begin; it != end; ++it) { - chassert(mutation_pointer < it->second->entry->znode_name); + /// FIXME uncomment this assertion after relesing 23.5 (currently it fails in Upgrade check) + /// chassert(mutation_pointer < it->second->entry->znode_name); mutation_ids.push_back(it->second->entry->znode_name); const auto & commands_from_entry = it->second->entry->commands; commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); diff --git a/tests/queries/0_stateless/02440_mutations_finalization.reference b/tests/queries/0_stateless/02440_mutations_finalization.reference index a8b9c2acdce..c4bad0a3806 100644 --- a/tests/queries/0_stateless/02440_mutations_finalization.reference +++ b/tests/queries/0_stateless/02440_mutations_finalization.reference @@ -2,4 +2,4 @@ 1 0000000000 UPDATE n = 2 WHERE n = 1 ['all_0_0_0'] 0 2 -0000000000 UPDATE n = 2 WHERE n = 1 [] 1 +0000000000 UPDATE n = 2 WHERE n = 1 [] diff --git a/tests/queries/0_stateless/02440_mutations_finalization.sql b/tests/queries/0_stateless/02440_mutations_finalization.sql index 796dcde8e4e..c522d8ab9df 100644 --- a/tests/queries/0_stateless/02440_mutations_finalization.sql +++ b/tests/queries/0_stateless/02440_mutations_finalization.sql @@ -6,6 +6,7 @@ system stop merges mut; alter table mut update n = 2 where n = 1; -- it will create MUTATE_PART entry, but will not execute it +system sync replica mut pull; select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; -- merges (and mutations) will start again after detach/attach, we need to avoid this somehow... @@ -26,8 +27,8 @@ select mutation_id, command, parts_to_do_names, is_done from system.mutations wh alter table mut modify setting max_number_of_mutations_for_replica=100; system sync replica mut; --- and now it should +-- and now it should (is_done may be 0, but it's okay) select * from mut; -select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; +select mutation_id, command, parts_to_do_names from system.mutations where database=currentDatabase() and table='mut'; drop table tmp; -- btw, it will check that mutation can be cancelled between blocks on shutdown diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql index d274fae1a4f..b9b1b645e8e 100644 --- a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql @@ -5,11 +5,17 @@ insert into mut values (1, 2, 3), (10, 20, 30); system stop merges mut; alter table mut delete where n = 10; + +-- a funny way to wait for a MUTATE_PART to be assigned +select sleepEachRow(2) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( + 'select 1 where ''MUTATE_PART'' not in (select type from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'')' + ), 'LineAsString', 's String') settings max_threads=1 format Null; + alter table mut drop column k settings alter_sync=0; system sync replica mut pull; -- a funny way to wait for ALTER_METADATA to disappear from the replication queue -select sleepEachRow(1) from url('http://localhost:8123/?param_tries={1..30}&query=' || encodeURLComponent( +select sleepEachRow(2) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( 'select * from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'' and type=''ALTER_METADATA''' ), 'LineAsString', 's String') settings max_threads=1 format Null; From b12eefc69444c7171ea3c0f2ba4c96bc51deece8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 16 May 2023 18:57:04 +0000 Subject: [PATCH 036/129] fix timeout units and log message --- src/Common/PoolBase.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index a9c595c440c..ba19c3e2150 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -146,12 +146,12 @@ public: if (timeout < 0) { - LOG_INFO(log, "No free connections in pool. Waiting undefinitelly."); + LOG_INFO(log, "No free connections in pool. Waiting indefinitely."); available.wait(lock); } else { - auto timeout_ms = std::chrono::microseconds(timeout); + auto timeout_ms = std::chrono::milliseconds(timeout); LOG_INFO(log, "No free connections in pool. Waiting {} ms.", timeout_ms.count()); available.wait_for(lock, timeout_ms); } From 64bf477ff29c6541d5c96c6ea333fb371d40f1c4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 16 May 2023 14:58:04 -0400 Subject: [PATCH 037/129] add DDL --- docs/en/operations/named-collections.md | 152 ++++++++++++++++++++---- 1 file changed, 127 insertions(+), 25 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 0189cc0a8e2..b65438b97f3 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -2,34 +2,89 @@ slug: /en/operations/named-collections sidebar_position: 69 sidebar_label: "Named collections" +title: "Named collections" --- -# Storing details for connecting to external sources in configuration files +Named collections provide a way to store collections of key-value pairs to be +used to configure integrations with external sources. You can use named collections with +dictionaries, tables, table functions, and object storage. + +Named collections can be configured with DDL or in configuration files and are applied +when ClickHouse starts. They simplify the creation of objects and the hiding of credentials +from users without administrative access. + +## Storing details for connecting to external sources in the system database + +### Permissions to create named collections with DDL + +To manage named collections with DDL a user must have the `named_control_collection` privilege. This can be assigned by adding a file to `/etc/clickhouse-server/users.d/`. The example gives the user `default` both the `access_management` and `named_collection_control` privileges: + +```xml title='/etc/clickhouse-server/users.d/user_default.xml' + + + + 65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5 + 1 + + 1 + + + + +``` + +:::tip +In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plaintext and sha256 hex passwords set for a user. +::: + +## Storing details for connecting to external sources in configuration files -Details for connecting to external sources (dictionaries, tables, table functions) can be saved -in configuration files and thus simplify the creation of objects and hide credentials -from users with only SQL access. Parameters can be set in XML `CSV` and overridden in SQL `, format = 'TSV'`. The parameters in SQL can be overridden using format `key` = `value`: `compression_method = 'gzip'`. -Named collections are stored in the `config.xml` file of the ClickHouse server in the `` section and are applied when ClickHouse starts. +Named collections are stored in the system table `system.named_collections` or in `config.xml` file of the ClickHouse server in the `` section and are applied when ClickHouse starts. + +### DDL example + +```sql +CREATE NAMED COLLECTION name AS +key_1 = 'value', +key_2 = 'value2', +url = 'https://connection.url/' +``` + +### XML example -Example of configuration: ```xml $ cat /etc/clickhouse-server/config.d/named_collections.xml - ... + + value + value_2 + https://connection.url/ + ``` -## Named collections for accessing S3. +## Named collections for accessing S3 The description of parameters see [s3 Table Function](../sql-reference/table-functions/s3.md). -Example of configuration: +### DDL example + +```sql +CREATE NAMED COLLECTION s3_mydata AS +access_key_id = 'AKIAIOSFODNN7EXAMPLE', +secret_access_key = 'wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY', +format = 'CSV', +url = 'https://s3.us-east-1.amazonaws.com/yourbucket/mydata/' +``` + +### XML example + ```xml @@ -43,23 +98,23 @@ Example of configuration: ``` -### Example of using named collections with the s3 function +### s3() function and S3 Table Named collection examples + +Both of the following examples use the same named collection `s3_mydata`: + +#### s3() function ```sql INSERT INTO FUNCTION s3(s3_mydata, filename = 'test_file.tsv.gz', format = 'TSV', structure = 'number UInt64', compression_method = 'gzip') SELECT * FROM numbers(10000); - -SELECT count() -FROM s3(s3_mydata, filename = 'test_file.tsv.gz') - -┌─count()─┐ -│ 10000 │ -└─────────┘ -1 rows in set. Elapsed: 0.279 sec. Processed 10.00 thousand rows, 90.00 KB (35.78 thousand rows/s., 322.02 KB/s.) ``` -### Example of using named collections with an S3 table +:::tip +The first argument to the `s3()` function above is the name of the collection, `s3_mydata`. Without named collections, the access key ID, secret, format, and URL would all be passed in every call to the `s3()` function. +::: + +#### S3 table ```sql CREATE TABLE s3_engine_table (number Int64) @@ -78,7 +133,22 @@ SELECT * FROM s3_engine_table LIMIT 3; The description of parameters see [mysql](../sql-reference/table-functions/mysql.md). -Example of configuration: +### DDL example + +```sql +CREATE NAMED COLLECTION mymysql AS +user = 'myuser', +password = 'mypass', +host = '127.0.0.1', +port = 3306, +database = 'test' +connection_pool_size = 8 +on_duplicate_clause = 1 +replace_query = 1 +``` + +### XML example + ```xml @@ -96,7 +166,11 @@ Example of configuration: ``` -### Example of using named collections with the mysql function +### mysql() function, MySQL table, MySQL database, and Dictionary named collection examples + +The four following examples use the same named collection `mymysql`: + +#### mysql() function ```sql SELECT count() FROM mysql(mymysql, table = 'test'); @@ -105,8 +179,11 @@ SELECT count() FROM mysql(mymysql, table = 'test'); │ 3 │ └─────────┘ ``` +:::note +The named collection does not specify the `table` parameter, so it is specified in the function call as `table = 'test'`. +::: -### Example of using named collections with an MySQL table +#### MySQL table ```sql CREATE TABLE mytable(A Int64) ENGINE = MySQL(mymysql, table = 'test', connection_pool_size=3, replace_query=0); @@ -117,7 +194,11 @@ SELECT count() FROM mytable; └─────────┘ ``` -### Example of using named collections with database with engine MySQL +:::note +The DDL overrides the named collection setting for connection_pool_size. +::: + +#### MySQL database ```sql CREATE DATABASE mydatabase ENGINE = MySQL(mymysql); @@ -130,7 +211,7 @@ SHOW TABLES FROM mydatabase; └────────┘ ``` -### Example of using named collections with a dictionary with source MySQL +#### MySQL Dictionary ```sql CREATE DICTIONARY dict (A Int64, B String) @@ -150,6 +231,17 @@ SELECT dictGet('dict', 'B', 2); The description of parameters see [postgresql](../sql-reference/table-functions/postgresql.md). +```sql +CREATE NAMED COLLECTION mypg AS +user = 'pguser', +password = 'jw8s0F4', +host = '127.0.0.1', +port = 5432, +database = 'test', +schema = 'test_schema', +connection_pool_size = 8 +``` + Example of configuration: ```xml @@ -229,12 +321,22 @@ SELECT dictGet('dict', 'b', 2); └─────────────────────────┘ ``` -## Named collections for accessing remote ClickHouse database +## Named collections for accessing a remote ClickHouse database The description of parameters see [remote](../sql-reference/table-functions/remote.md/#parameters). Example of configuration: +```sql +CREATE NAMED COLLECTION remote1 AS +host = 'remote_host', +port = 9000, +database = 'system', +user = 'foo', +password = 'secret', +secure = 1 +``` + ```xml From abacf1f99088ee52b46173537a65aab6b0e6306c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 16 May 2023 19:14:54 +0000 Subject: [PATCH 038/129] add missing `quota_key` in operator== for connections --- src/Client/ConnectionPool.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index c3d0955019e..9b1f7dc5b9d 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -192,6 +192,7 @@ inline bool operator==(const ConnectionPoolFactory::Key & lhs, const ConnectionP { return lhs.max_connections == rhs.max_connections && lhs.host == rhs.host && lhs.port == rhs.port && lhs.default_database == rhs.default_database && lhs.user == rhs.user && lhs.password == rhs.password + && lhs.quota_key == rhs.quota_key && lhs.cluster == rhs.cluster && lhs.cluster_secret == rhs.cluster_secret && lhs.client_name == rhs.client_name && lhs.compression == rhs.compression && lhs.secure == rhs.secure && lhs.priority == rhs.priority; } From 35f00f72b3bad1025453181f785c20c9945dda36 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 22:42:43 +0200 Subject: [PATCH 039/129] add functional test --- tests/queries/0_stateless/02751_match_constant_needle.reference | 1 + tests/queries/0_stateless/02751_match_constant_needle.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02751_match_constant_needle.reference create mode 100644 tests/queries/0_stateless/02751_match_constant_needle.sql diff --git a/tests/queries/0_stateless/02751_match_constant_needle.reference b/tests/queries/0_stateless/02751_match_constant_needle.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02751_match_constant_needle.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02751_match_constant_needle.sql b/tests/queries/0_stateless/02751_match_constant_needle.sql new file mode 100644 index 00000000000..71bdcc7cb0a --- /dev/null +++ b/tests/queries/0_stateless/02751_match_constant_needle.sql @@ -0,0 +1 @@ +select match('default/k8s1', '\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z'); From c10435489487626448b8e861074ef9b41ecb9f4e Mon Sep 17 00:00:00 2001 From: fhbai Date: Wed, 17 May 2023 14:39:30 +0800 Subject: [PATCH 040/129] fix --- src/Functions/array/arrayDotProduct.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index e3c80775f1b..d17c223cc2f 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -57,7 +57,7 @@ public: } template - static ResultType apply( + static inline NO_SANITIZE_UNDEFINED ResultType apply( const T * left, const U * right, size_t size) From 3c80e30f02f40acd372942506d8d57ec803ef2b6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 16:59:19 +0200 Subject: [PATCH 041/129] Fix per-query IO/BACKUPs throttling settings (when default profile has them) When some of this settings was set for default profile (in users.xml/users.yml), then it will be always used regardless of what user passed. Fix this by not inherit per-query throttlers, for this they should be reset before making query context and they should not be initialized as before in Context::makeQueryContext(), since makeQueryContext() called too early, when user settings was not read yet. But there we had also initialization of per-server throttling, move this into the ContextSharedPart::configureServerWideThrottling(), and call it once we have ServerSettings set. Also note, that this patch makes the following settings - server settings: - max_replicated_fetches_network_bandwidth_for_server - max_replicated_sends_network_bandwidth_for_server But this change should not affect anybody, since it is done with compatiblity (i.e. if this setting is set in users profile it will be read from it as well as a fallback). Signed-off-by: Azat Khuzhin --- src/Core/ServerSettings.h | 2 + src/Core/Settings.h | 4 +- src/Interpreters/Context.cpp | 150 +++++++----------- .../configs/limit_replication_config.xml | 8 +- .../test_replicated_fetches_bandwidth/test.py | 2 +- 5 files changed, 65 insertions(+), 101 deletions(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index ee3482414af..2a73930836a 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -23,6 +23,8 @@ namespace DB M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \ M(UInt64, outdated_part_loading_thread_pool_queue_size, 10000, "Queue size for parts loading thread pool.", 0) \ + M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ + M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ M(UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0) \ M(UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 610c7135a75..874e31b8d37 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -101,8 +101,6 @@ class IColumn; M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ M(Bool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ - M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.", 0) \ - M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited. Only has meaning at server startup.", 0) \ M(UInt64, max_remote_read_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for read.", 0) \ M(UInt64, max_remote_write_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for write.", 0) \ M(UInt64, max_local_read_bandwidth, 0, "The maximum speed of local reads in bytes per second.", 0) \ @@ -791,6 +789,8 @@ class IColumn; MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_distributed_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_read_network_bandwidth_for_server, 0) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_write_network_bandwidth_for_server, 0) \ + MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0) \ + MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0) \ /* ---- */ \ MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b4bdb7cf233..0ef9ea53ee8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -631,6 +631,30 @@ struct ContextSharedPart : boost::noncopyable log->warning(message); warnings.push_back(message); } + + void configureServerWideThrottling() + { + if (auto bandwidth = server_settings.max_replicated_fetches_network_bandwidth_for_server) + replicated_fetches_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_replicated_sends_network_bandwidth_for_server) + replicated_sends_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_remote_read_network_bandwidth_for_server) + remote_read_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_remote_write_network_bandwidth_for_server) + remote_write_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_local_read_bandwidth_for_server) + local_read_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_local_write_bandwidth_for_server) + local_write_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_backup_bandwidth_for_server) + backups_server_throttler = std::make_shared(bandwidth); + } }; @@ -1897,16 +1921,22 @@ void Context::makeQueryContext() { query_context = shared_from_this(); - /// Create throttlers, to inherit the ThrottlePtr in the context copies. - { - getRemoteReadThrottler(); - getRemoteWriteThrottler(); - - getLocalReadThrottler(); - getLocalWriteThrottler(); - - getBackupsThrottler(); - } + /// Throttling should not be inherited, otherwise if you will set + /// throttling for default profile you will not able to overwrite it + /// per-user/query. + /// + /// Note, that if you need to set it server-wide, you should use + /// per-server settings, i.e.: + /// - max_backup_bandwidth_for_server + /// - max_remote_read_network_bandwidth_for_server + /// - max_remote_write_network_bandwidth_for_server + /// - max_local_read_bandwidth_for_server + /// - max_local_write_bandwidth_for_server + remote_read_query_throttler.reset(); + remote_write_query_throttler.reset(); + local_read_query_throttler.reset(); + local_write_query_throttler.reset(); + backups_query_throttler.reset(); } void Context::makeSessionContext() @@ -2438,143 +2468,76 @@ BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const ThrottlerPtr Context::getReplicatedFetchesThrottler() const { - auto lock = getLock(); - if (!shared->replicated_fetches_throttler) - shared->replicated_fetches_throttler = std::make_shared( - settings.max_replicated_fetches_network_bandwidth_for_server); - return shared->replicated_fetches_throttler; } ThrottlerPtr Context::getReplicatedSendsThrottler() const { - auto lock = getLock(); - if (!shared->replicated_sends_throttler) - shared->replicated_sends_throttler = std::make_shared( - settings.max_replicated_sends_network_bandwidth_for_server); - return shared->replicated_sends_throttler; } ThrottlerPtr Context::getRemoteReadThrottler() const { - ThrottlerPtr throttler; - - const auto & query_settings = getSettingsRef(); - UInt64 bandwidth_for_server = shared->server_settings.max_remote_read_network_bandwidth_for_server; - if (bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->remote_read_throttler) - shared->remote_read_throttler = std::make_shared(bandwidth_for_server); - throttler = shared->remote_read_throttler; - } - - if (query_settings.max_remote_read_network_bandwidth) + ThrottlerPtr throttler = shared->remote_read_throttler; + if (auto bandwidth = getSettingsRef().max_remote_read_network_bandwidth) { auto lock = getLock(); if (!remote_read_query_throttler) - remote_read_query_throttler = std::make_shared(query_settings.max_remote_read_network_bandwidth, throttler); + remote_read_query_throttler = std::make_shared(bandwidth, throttler); throttler = remote_read_query_throttler; } - return throttler; } ThrottlerPtr Context::getRemoteWriteThrottler() const { - ThrottlerPtr throttler; - - const auto & query_settings = getSettingsRef(); - UInt64 bandwidth_for_server = shared->server_settings.max_remote_write_network_bandwidth_for_server; - if (bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->remote_write_throttler) - shared->remote_write_throttler = std::make_shared(bandwidth_for_server); - throttler = shared->remote_write_throttler; - } - - if (query_settings.max_remote_write_network_bandwidth) + ThrottlerPtr throttler = shared->remote_write_throttler; + if (auto bandwidth = getSettingsRef().max_remote_write_network_bandwidth) { auto lock = getLock(); if (!remote_write_query_throttler) - remote_write_query_throttler = std::make_shared(query_settings.max_remote_write_network_bandwidth, throttler); + remote_write_query_throttler = std::make_shared(bandwidth, throttler); throttler = remote_write_query_throttler; } - return throttler; } ThrottlerPtr Context::getLocalReadThrottler() const { - ThrottlerPtr throttler; - - if (shared->server_settings.max_local_read_bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->local_read_throttler) - shared->local_read_throttler = std::make_shared(shared->server_settings.max_local_read_bandwidth_for_server); - throttler = shared->local_read_throttler; - } - - const auto & query_settings = getSettingsRef(); - if (query_settings.max_local_read_bandwidth) + ThrottlerPtr throttler = shared->local_read_throttler; + if (auto bandwidth = getSettingsRef().max_local_read_bandwidth) { auto lock = getLock(); if (!local_read_query_throttler) - local_read_query_throttler = std::make_shared(query_settings.max_local_read_bandwidth, throttler); + local_read_query_throttler = std::make_shared(bandwidth, throttler); throttler = local_read_query_throttler; } - return throttler; } ThrottlerPtr Context::getLocalWriteThrottler() const { - ThrottlerPtr throttler; - - if (shared->server_settings.max_local_write_bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->local_write_throttler) - shared->local_write_throttler = std::make_shared(shared->server_settings.max_local_write_bandwidth_for_server); - throttler = shared->local_write_throttler; - } - - const auto & query_settings = getSettingsRef(); - if (query_settings.max_local_write_bandwidth) + ThrottlerPtr throttler = shared->local_write_throttler; + if (auto bandwidth = getSettingsRef().max_local_write_bandwidth) { auto lock = getLock(); if (!local_write_query_throttler) - local_write_query_throttler = std::make_shared(query_settings.max_local_write_bandwidth, throttler); + local_write_query_throttler = std::make_shared(bandwidth, throttler); throttler = local_write_query_throttler; } - return throttler; } ThrottlerPtr Context::getBackupsThrottler() const { - ThrottlerPtr throttler; - - if (shared->server_settings.max_backup_bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->backups_server_throttler) - shared->backups_server_throttler = std::make_shared(shared->server_settings.max_backup_bandwidth_for_server); - throttler = shared->backups_server_throttler; - } - - const auto & query_settings = getSettingsRef(); - if (query_settings.max_backup_bandwidth) + ThrottlerPtr throttler = shared->backups_server_throttler; + if (auto bandwidth = getSettingsRef().max_backup_bandwidth) { auto lock = getLock(); if (!backups_query_throttler) - backups_query_throttler = std::make_shared(query_settings.max_backup_bandwidth, throttler); + backups_query_throttler = std::make_shared(bandwidth, throttler); throttler = backups_query_throttler; } - return throttler; } @@ -3633,7 +3596,10 @@ void Context::setApplicationType(ApplicationType type) shared->application_type = type; if (type == ApplicationType::SERVER) + { shared->server_settings.loadSettingsFromConfig(Poco::Util::Application::instance().config()); + shared->configureServerWideThrottling(); + } } void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) diff --git a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml index b18f0fbc93f..2a7b47e3560 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml +++ b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml @@ -1,8 +1,4 @@ - - - 5242880 - 10485760 - - + 5242880 + 10485760 diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py index 059102f8683..cd969746c31 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/test.py +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -12,7 +12,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", with_zookeeper=True) node2 = cluster.add_instance("node2", with_zookeeper=True) node3 = cluster.add_instance( - "node3", user_configs=["configs/limit_replication_config.xml"], with_zookeeper=True + "node3", main_configs=["configs/limit_replication_config.xml"], with_zookeeper=True ) From 7383da0c526399963ff3496c22b6a59f062ff98f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 21:43:43 +0200 Subject: [PATCH 042/129] Fix per-query remote throttler remote throttler by some reason had been overwritten by the global one during reloads, likely this is for graceful reload of this option, but it breaks per-query throttling, remove this logic. Signed-off-by: Azat Khuzhin --- .../AzureBlobStorage/AzureObjectStorage.cpp | 1 - src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 5 ----- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h | 5 ----- src/Disks/ObjectStorages/IObjectStorage.cpp | 11 ----------- src/Disks/ObjectStorages/IObjectStorage.h | 11 ++++------- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 1 - 6 files changed, 4 insertions(+), 30 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 9db5d13a7f8..62c3216ad3f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -282,7 +282,6 @@ void AzureObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguratio { auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context); settings.set(std::move(new_settings)); - applyRemoteThrottlingSettings(context); /// We don't update client } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index e50e410823d..38c088ab213 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -156,11 +156,6 @@ void HDFSObjectStorage::copyObject( /// NOLINT } -void HDFSObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr context) -{ - applyRemoteThrottlingSettings(context); -} - std::unique_ptr HDFSObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS object storage doesn't support cloning"); diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 4064a5c5b7f..fdc47ad16a6 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -105,11 +105,6 @@ public: void startup() override; - void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - String getObjectsNamespace() const override { return ""; } std::unique_ptr cloneObjectStorage( diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 52e8b1a465d..1ee55a7b342 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -55,27 +55,16 @@ const std::string & IObjectStorage::getCacheName() const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getCacheName() is not implemented for object storage"); } -void IObjectStorage::applyRemoteThrottlingSettings(ContextPtr context) -{ - std::unique_lock lock{throttlers_mutex}; - remote_read_throttler = context->getRemoteReadThrottler(); - remote_write_throttler = context->getRemoteWriteThrottler(); -} - ReadSettings IObjectStorage::patchSettings(const ReadSettings & read_settings) const { - std::unique_lock lock{throttlers_mutex}; ReadSettings settings{read_settings}; - settings.remote_throttler = remote_read_throttler; settings.for_object_storage = true; return settings; } WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings) const { - std::unique_lock lock{throttlers_mutex}; WriteSettings settings{write_settings}; - settings.remote_throttler = remote_write_throttler; settings.for_object_storage = true; return settings; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index f4f1b063ade..58a31fdc8c3 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -165,9 +165,10 @@ public: /// Apply new settings, in most cases reiniatilize client and some other staff virtual void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) = 0; + const Poco::Util::AbstractConfiguration &, + const std::string & /*config_prefix*/, + ContextPtr) + {} /// Sometimes object storages have something similar to chroot or namespace, for example /// buckets in S3. If object storage doesn't have any namepaces return empty string. @@ -205,10 +206,6 @@ public: virtual WriteSettings patchSettings(const WriteSettings & write_settings) const; -protected: - /// Should be called from implementation of applyNewSettings() - void applyRemoteThrottlingSettings(ContextPtr context); - private: mutable std::mutex throttlers_mutex; ThrottlerPtr remote_read_throttler; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79b3d3a2b8b..9faae3a6c62 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -443,7 +443,6 @@ void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & auto new_client = getClient(config, config_prefix, context, *new_s3_settings); s3_settings.set(std::move(new_s3_settings)); client.set(std::move(new_client)); - applyRemoteThrottlingSettings(context); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( From fdfb1eda55b0c2ee95dc7284629f7ded6f4c9196 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 22:15:05 +0200 Subject: [PATCH 043/129] Fix {Local,Remote}ReadThrottlerSleepMicroseconds metric values And also update the test, since now you could have slightly less sleep intervals, if query spend some time in other places. But what is important is that query_duration_ms does not exceeded calculated delay. Signed-off-by: Azat Khuzhin --- src/Common/Throttler.cpp | 16 ++++++++-------- src/Common/Throttler.h | 8 ++++---- .../02703_max_local_read_bandwidth.sh | 2 +- .../02703_max_local_write_bandwidth.sh | 2 +- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 4f99c24cc8d..4c1320db27a 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -61,20 +61,20 @@ UInt64 Throttler::add(size_t amount) throw Exception::createDeprecated(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); /// Wait unless there is positive amount of tokens - throttling - Int64 sleep_time = 0; + Int64 sleep_time_ns = 0; if (max_speed && tokens_value < 0) { - sleep_time = static_cast(-tokens_value / max_speed * NS); - accumulated_sleep += sleep_time; - sleepForNanoseconds(sleep_time); - accumulated_sleep -= sleep_time; - ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time / 1000UL); + sleep_time_ns = static_cast(-tokens_value / max_speed * NS); + accumulated_sleep += sleep_time_ns; + sleepForNanoseconds(sleep_time_ns); + accumulated_sleep -= sleep_time_ns; + ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time_ns / 1000UL); } if (parent) - sleep_time += parent->add(amount); + sleep_time_ns += parent->add(amount); - return static_cast(sleep_time); + return static_cast(sleep_time_ns); } void Throttler::reset() diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 4b117ae7637..7508065096b 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -34,15 +34,15 @@ public: const std::shared_ptr & parent_ = nullptr); /// Use `amount` tokens, sleeps if required or throws exception on limit overflow. - /// Returns duration of sleep in microseconds (to distinguish sleeping on different kinds of throttlers for metrics) + /// Returns duration of sleep in nanoseconds (to distinguish sleeping on different kinds of throttlers for metrics) UInt64 add(size_t amount); UInt64 add(size_t amount, ProfileEvents::Event event_amount, ProfileEvents::Event event_sleep_us) { - UInt64 sleep_us = add(amount); + UInt64 sleep_ns = add(amount); ProfileEvents::increment(event_amount, amount); - ProfileEvents::increment(event_sleep_us, sleep_us); - return sleep_us; + ProfileEvents::increment(event_sleep_us, sleep_ns / 1000UL); + return sleep_ns; } /// Not thread safe diff --git a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh index 130f3a29ade..d47e2f363bd 100755 --- a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh @@ -32,7 +32,7 @@ for read_method in "${read_methods[@]}"; do query_duration_ms >= 7e3, ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] > 8e6, ProfileEvents['LocalReadThrottlerBytes'] > 8e6, - ProfileEvents['LocalReadThrottlerSleepMicroseconds'] > 7e6 + ProfileEvents['LocalReadThrottlerSleepMicroseconds'] > 7e6*0.9 FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' " diff --git a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh index 80713e90169..41165d35d37 100755 --- a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -nm -q " query_duration_ms >= 7e3, ProfileEvents['WriteBufferFromFileDescriptorWriteBytes'] > 8e6, ProfileEvents['LocalWriteThrottlerBytes'] > 8e6, - ProfileEvents['LocalWriteThrottlerSleepMicroseconds'] > 7e6 + ProfileEvents['LocalWriteThrottlerSleepMicroseconds'] > 7e6*0.9 FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' " From 9fe4f1a934c86d5d42fef6bbd7c3d19db4e7f97b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 21:58:07 +0200 Subject: [PATCH 044/129] Extensive coverage for bandwidth limiting settings Signed-off-by: Azat Khuzhin --- tests/integration/test_throttling/__init__.py | 0 .../configs/server_backups.xml | 34 ++ .../configs/server_overrides.xml | 3 + .../configs/users_overrides.xml | 3 + tests/integration/test_throttling/test.py | 413 ++++++++++++++++++ 5 files changed, 453 insertions(+) create mode 100644 tests/integration/test_throttling/__init__.py create mode 100644 tests/integration/test_throttling/configs/server_backups.xml create mode 100644 tests/integration/test_throttling/configs/server_overrides.xml create mode 100644 tests/integration/test_throttling/configs/users_overrides.xml create mode 100644 tests/integration/test_throttling/test.py diff --git a/tests/integration/test_throttling/__init__.py b/tests/integration/test_throttling/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_throttling/configs/server_backups.xml b/tests/integration/test_throttling/configs/server_backups.xml new file mode 100644 index 00000000000..d25c67a779c --- /dev/null +++ b/tests/integration/test_throttling/configs/server_backups.xml @@ -0,0 +1,34 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + +
+ s3 +
+
+
+
+
+ + + + http://minio1:9001/root/data/ + minio + minio123 + + + + + default + /backups/ + +
diff --git a/tests/integration/test_throttling/configs/server_overrides.xml b/tests/integration/test_throttling/configs/server_overrides.xml new file mode 100644 index 00000000000..197bf660500 --- /dev/null +++ b/tests/integration/test_throttling/configs/server_overrides.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_throttling/configs/users_overrides.xml b/tests/integration/test_throttling/configs/users_overrides.xml new file mode 100644 index 00000000000..197bf660500 --- /dev/null +++ b/tests/integration/test_throttling/configs/users_overrides.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py new file mode 100644 index 00000000000..ff8e7154d0d --- /dev/null +++ b/tests/integration/test_throttling/test.py @@ -0,0 +1,413 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +# This test covers the following options: +# - max_backup_bandwidth +# - max_backup_bandwidth_for_server +# - max_local_read_bandwidth +# - max_local_read_bandwidth_for_server +# - max_local_write_bandwidth +# - max_local_write_bandwidth_for_server +# - max_remote_read_network_bandwidth +# - max_remote_read_network_bandwidth_for_server +# - max_remote_write_network_bandwidth +# - max_remote_write_network_bandwidth_for_server +# - and that max_backup_bandwidth from the query will override setting from the user profile + +import time +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + + +def elapsed(func, *args, **kwargs): + start = time.time() + ret = func(*args, **kwargs) + end = time.time() + return ret, end - start + + +node = cluster.add_instance( + "node", + stay_alive=True, + main_configs=["configs/server_backups.xml", "configs/server_overrides.xml"], + user_configs=["configs/users_overrides.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield + finally: + cluster.shutdown() + + +@pytest.fixture(scope="function", autouse=True) +def revert_config(): + # Revert configs after the test, not before + yield + node.exec_in_container( + [ + "bash", + "-c", + f"echo '' > /etc/clickhouse-server/config.d/server_overrides.xml", + ] + ) + node.exec_in_container( + [ + "bash", + "-c", + f"echo '' > /etc/clickhouse-server/users.d/users_overrides.xml", + ] + ) + node.restart_clickhouse() + + +backup_id_counter = 0 + + +def next_backup_name(storage): + global backup_id_counter + if storage == "local": + backup_id_counter += 1 + return f"Disk('default', '{backup_id_counter}/')" + elif storage == "remote": + backup_id_counter += 1 + return f"S3(s3, '{backup_id_counter}/')" + else: + raise Exception(storage) + + +def node_update_config(mode, setting, value=None): + if mode is None: + return + if mode == "server": + config_path = "/etc/clickhouse-server/config.d/server_overrides.xml" + config_content = f""" + <{setting}>{value} + """ + else: + config_path = "/etc/clickhouse-server/users.d/users_overrides.xml" + config_content = f""" + + + + <{setting}>{value} + + + + """ + node.exec_in_container( + [ + "bash", + "-c", + f"echo '{config_content}' > {config_path}", + ] + ) + node.restart_clickhouse() + + +def assert_took(took, should_took): + assert took >= should_took[0] * 0.9 and took < should_took[1] + + +@pytest.mark.parametrize( + "policy,backup_name,mode,setting,value,should_took", + [ + # + # Local -> Local + # + pytest.param( + "default", + next_backup_name("local"), + None, + None, + None, + (0, 3), + id="no_local_throttling", + ), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "default", + next_backup_name("local"), + "user", + "max_backup_bandwidth", + "1M", + (7, 14), + id="user_local_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "default", + next_backup_name("local"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (3, 7), + id="server_local_throttling", + ), + # + # Remote -> Local + # + pytest.param( + "s3", + next_backup_name("local"), + None, + None, + None, + (0, 3), + id="no_remote_to_local_throttling", + ), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "s3", + next_backup_name("local"), + "user", + "max_backup_bandwidth", + "1M", + (7, 14), + id="user_remote_to_local_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "s3", + next_backup_name("local"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (3, 7), + id="server_remote_to_local_throttling", + ), + # + # Remote -> Remote + # + pytest.param( + "s3", + next_backup_name("remote"), + None, + None, + None, + (0, 3), + id="no_remote_to_remote_throttling", + ), + # No throttling for S3-to-S3, uses native copy + pytest.param( + "s3", + next_backup_name("remote"), + "user", + "max_backup_bandwidth", + "1M", + (0, 3), + id="user_remote_to_remote_throttling", + ), + # No throttling for S3-to-S3, uses native copy + pytest.param( + "s3", + next_backup_name("remote"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (0, 3), + id="server_remote_to_remote_throttling", + ), + # + # Local -> Remote + # + # NOTE: S3 is complex, it will read file 3 times: + # - first for calculating the checksum + # - second for calculating the signature + # - and finally to write the payload to S3 + # Hence the value should be multipled by 3. + pytest.param( + "default", + next_backup_name("remote"), + None, + None, + None, + (0, 3), + id="no_local_to_remote_throttling", + ), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds, but for S3Client it is 2x more + pytest.param( + "default", + next_backup_name("remote"), + "user", + "max_backup_bandwidth", + "1M", + (7 * 3, 7 * 4 - 1), + id="user_local_to_remote_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds, but for S3Client it is 2x more + pytest.param( + "default", + next_backup_name("remote"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (3 * 3, 3 * 5), + id="server_local_to_remote_throttling", + ), + ], +) +def test_backup_throttling(policy, backup_name, mode, setting, value, should_took): + node_update_config(mode, setting, value) + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='{policy}'; + insert into data select * from numbers(1e6); + """ + ) + _, took = elapsed(node.query, f"backup table data to {backup_name}") + assert_took(took, should_took) + + +def test_backup_throttling_override(): + node_update_config("user", "max_backup_bandwidth", "1M") + node.query( + """ + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; + insert into data select * from numbers(1e6); + """ + ) + + backup_name = next_backup_name("local") + _, took = elapsed( + node.query, + f"backup table data to {backup_name}", + settings={ + "max_backup_bandwidth": "500K", + }, + ) + # reading 1e6*8 bytes with 500Ki default bandwith should take (8-0.5)/0.5=15 seconds + assert_took(took, (15, 20)) + + +@pytest.mark.parametrize( + "policy,mode,setting,value,should_took", + [ + # + # Local + # + pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "default", + "user", + "max_local_read_bandwidth", + "1M", + (7, 14), + id="user_local_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "default", + "server", + "max_local_read_bandwidth_for_server", + "2M", + (3, 7), + id="server_local_throttling", + ), + # + # Remote + # + pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "s3", + "user", + "max_remote_read_network_bandwidth", + "1M", + (7, 14), + id="user_remote_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "s3", + "server", + "max_remote_read_network_bandwidth_for_server", + "2M", + (3, 7), + id="server_remote_throttling", + ), + ], +) +def test_read_throttling(policy, mode, setting, value, should_took): + node_update_config(mode, setting, value) + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='{policy}'; + insert into data select * from numbers(1e6); + """ + ) + _, took = elapsed(node.query, f"select * from data") + assert_took(took, should_took) + + +@pytest.mark.parametrize( + "policy,mode,setting,value,should_took", + [ + # + # Local + # + pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "default", + "user", + "max_local_write_bandwidth", + "1M", + (7, 14), + id="local_user_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "default", + "server", + "max_local_write_bandwidth_for_server", + "2M", + (3, 7), + id="local_server_throttling", + ), + # + # Remote + # + pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + # writeing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "s3", + "user", + "max_remote_write_network_bandwidth", + "1M", + (7, 14), + id="user_remote_throttling", + ), + # writeing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "s3", + "server", + "max_remote_write_network_bandwidth_for_server", + "2M", + (3, 7), + id="server_remote_throttling", + ), + ], +) +def test_write_throttling(policy, mode, setting, value, should_took): + node_update_config(mode, setting, value) + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='{policy}'; + """ + ) + _, took = elapsed(node.query, f"insert into data select * from numbers(1e6)") + assert_took(took, should_took) From 3d8b6b708fd655c4012afd1702092a36a3fb2e86 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 May 2023 10:20:29 +0000 Subject: [PATCH 045/129] Fix for analyzer: 02377_ ptimize_sorting_by_input_stream_properties_explain --- ..._input_stream_properties_explain.reference | 74 +++++++++++++++++++ ...ting_by_input_stream_properties_explain.sh | 5 +- 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index 0b34e878cee..69c325c21a9 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -5,6 +5,12 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (None) Sorting (None) +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (None) +Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort -- QUERY: set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=3;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a MergingSortedTransform 3 → 1 @@ -27,12 +33,24 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Stream): a ASC Sorting (Stream): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Stream): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 +Sorting (None) +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC @@ -40,28 +58,54 @@ Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (None) Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 +Sorting (Global): plus(a, 1) ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC +Sorting (None) +Sorting (Chunk): a ASC -- FilterStep preserves sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 +Sorting (Chunk): a ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 +Sorting (Chunk): a ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 +Sorting (Chunk): a ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- FilterStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 +Sorting (None) +Sorting (None) +Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 +Sorting (None) +Sorting (None) +Sorting (Chunk): a ASC -- aliases break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a Sorting (Global): a ASC @@ -72,6 +116,15 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Stream): a ASC Sorting (Stream): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): a_0 ASC +Sorting (None) +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_2 ASC +Sorting (Stream): default.optimize_sorting.a_2 ASC +Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) Sorting (Global): a ASC, b ASC @@ -79,6 +132,12 @@ Sorting (Sorting for ORDER BY) Sorting (Global): x ASC, y ASC Sorting (Stream): x ASC, y ASC Sorting (Stream): a ASC, b ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) +Sorting (Global): a ASC, b ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): x_2 ASC, y_3 ASC +Sorting (Stream): x_2 ASC, y_3 ASC +Sorting (Stream): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) @@ -89,6 +148,15 @@ Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 +Sorting (None) +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(a_0, 1_UInt8) ASC +Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC +Sorting (Chunk): default.optimize_sorting.a_3 ASC +Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC @@ -96,3 +164,9 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Stream): a ASC Sorting (Stream): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Stream): a ASC diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index e4cab987d35..7e937ac42b6 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -17,9 +17,12 @@ function explain_sorting { echo "-- QUERY: "$1 $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTING } + function explain_sortmode { echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTMODE + $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -nq "$1" | eval $FIND_SORTMODE + echo "-- QUERY (analyzer): "$1 + $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -nq "$1" | eval $FIND_SORTMODE } $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" From a1fc96953f41818a1e661af49f14d4cad8d7226f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 17 May 2023 14:48:16 -0400 Subject: [PATCH 046/129] reorder --- docs/en/operations/named-collections.md | 37 +++++++++++++------------ 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index b65438b97f3..1866315529e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -13,7 +13,23 @@ Named collections can be configured with DDL or in configuration files and are a when ClickHouse starts. They simplify the creation of objects and the hiding of credentials from users without administrative access. -## Storing details for connecting to external sources in the system database +The keys in a named collection must match the parameter names of the corresponding +function, table engine, database, etc. In the examples below the parameter list is +linked to for each type. + +Parameters set in a named collection can be overridden in SQL, this is shown in the examples +below. + +## Storing named collections in the system database + +### DDL example + +```sql +CREATE NAMED COLLECTION name AS +key_1 = 'value', +key_2 = 'value2', +url = 'https://connection.url/' +``` ### Permissions to create named collections with DDL @@ -37,22 +53,7 @@ To manage named collections with DDL a user must have the `named_control_collect In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plaintext and sha256 hex passwords set for a user. ::: -## Storing details for connecting to external sources in configuration files - - -Parameters can be set in XML `CSV` and overridden in SQL `, format = 'TSV'`. -The parameters in SQL can be overridden using format `key` = `value`: `compression_method = 'gzip'`. - -Named collections are stored in the system table `system.named_collections` or in `config.xml` file of the ClickHouse server in the `` section and are applied when ClickHouse starts. - -### DDL example - -```sql -CREATE NAMED COLLECTION name AS -key_1 = 'value', -key_2 = 'value2', -url = 'https://connection.url/' -``` +## Storing named collections in configuration files ### XML example @@ -98,7 +99,7 @@ url = 'https://s3.us-east-1.amazonaws.com/yourbucket/mydata/'
``` -### s3() function and S3 Table Named collection examples +### s3() function and S3 Table named collection examples Both of the following examples use the same named collection `s3_mydata`: From 549af4d35112f598f3bbaaf196d0d994ff9076dc Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 May 2023 21:23:32 +0200 Subject: [PATCH 047/129] address comments --- docs/en/sql-reference/dictionaries/index.md | 50 +++++++++---------- ...4_regexp_dictionary_table_source.reference | 40 +++++++-------- .../02504_regexp_dictionary_table_source.sql | 14 +++--- 3 files changed, 50 insertions(+), 54 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 4abc41cdf42..eb45247e74a 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2197,13 +2197,13 @@ Result: └─────────────────────────────────┴───────┘ ``` -## RegExpTree Dictionary {#regexp-tree-dictionary} +## Regular Expression Tree Dictionary {#regexp-tree-dictionary} -RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some scenarios, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environments. +Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of (user agent)[https://en.wikipedia.org/wiki/User_agent] strings, which can be expressed elegantly with regexp tree dictionaries. -### Use RegExpTree Dictionary in local environment +### Use Regular Expression Tree Dictionary in ClickHouse Open-Source Environment -In local environment, we create RegexpTree dictionary by a yaml file: +Regular expression tree dictionaries are defined in ClickHouse open-source using the YAMLRegExpTree source which is provided the path to a YAML file containing the regular expression tree. ```sql create dictionary regexp_dict @@ -2218,7 +2218,7 @@ LAYOUT(regexp_tree) ... ``` -The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree. For example: +The dictionary source `YAMLRegExpTree` represents the structure of a regexp tree. For example: ```yaml - regexp: 'Linux/(\d+[\.\d]*).+tlinux' @@ -2226,7 +2226,7 @@ The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree version: '\1' - regexp: '\d+/tclwebkit(?:\d+[\.\d]*)' - name: 'Andriod' + name: 'Android' versions: - regexp: '33/tclwebkit' version: '13' @@ -2238,15 +2238,15 @@ The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree version: '10' ``` -This config consists of a list of RegExpTree nodes. Each node has following structure: +This config consists of a list of Regular Expression Tree nodes. Each node has following structure: - **regexp** means the regular expression of this node. -- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have both attributions. The second node only has `name` attribution, because the `version` is defined in the children nodes. - - The value of an attribution could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. -- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, children nodes have more priority. - - the name of **children nodes** in yaml files can be arbitrary. +- **user defined attributes** is a list of dictionary attributes defined in the dictionary structure. In this case, we have two attributes: `name` and `version`. The first node has both attributes. The second node only has `name` attribute, because the `version` is defined in the children nodes. + - The value of an attribute could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. +- **children nodes** is the children of a regexp tree node, which has their own attributes and children nodes. String matching preceeds in a depth-first fasion. If a string matches any regexp node in the top layer, the dictionary checks if the string matches the children nodes of it. If it matches, we assign the attributes of the matching nodes. If two or more nodes define the same attribute, children nodes have more priority. + - the name of **children nodes** in YAML files can be arbitrary. -Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. +Due to the specialty of regexp tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. Example: @@ -2258,17 +2258,17 @@ Result: ```text ┌─dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024')─┐ -│ ('Andriod','12') │ +│ ('Android','12') │ └─────────────────────────────────────────────────────────────────┘ ``` -Explain: +In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node, so the dictionary continues to look into the children nodes and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Android` defined in the first layer and the value of `version` is `12` defined the child node. -In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the first layer, so the dictionary will continue to look into the children nodes in the second layer and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Andriod` defined in the first layer and the value of `version` is `12` defined in the second layer. +With a powerful YAML configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) -### Use RegExpTree Dictionary on cloud +### Use Regular Expression Tree Dictionary in ClickHouse Cloud -We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. +We have shown how Regular Expression Tree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local YAML file, we can use this file to create Regular Expression Tree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. ```sql select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') @@ -2278,7 +2278,7 @@ The content of csv file is: ```text 1,0,"Linux/(\d+[\.\d]*).+tlinux","['version','name']","['\\1','TencentOS']" -2,0,"(\d+)/tclwebkit(\d+[\.\d]*)","['comment','version','name']","['test $1 and $2','$1','Andriod']" +2,0,"(\d+)/tclwebkit(\d+[\.\d]*)","['comment','version','name']","['test $1 and $2','$1','Android']" 3,2,"33/tclwebkit","['version']","['13']" 4,2,"3[12]/tclwebkit","['version']","['12']" 5,2,"3[12]/tclwebkit","['version']","['11']" @@ -2287,11 +2287,11 @@ The content of csv file is: The schema of dumped file is always -- `id UInt64` represents the identify number of the RegexpTree node. +- `id UInt64` represents the id of the RegexpTree node. - `parent_id UInt64` represents the id of the parent of a node. - `regexp String` represents the regular expression string. -- `keys Array(String)` represents the names of user defined attributions. -- `values Array(String)` represents the values of user defined attributions. +- `keys Array(String)` represents the names of user defined attributes. +- `values Array(String)` represents the values of user defined attributes. On the cloud, we can create a table `regexp_dictionary_source_table` with the above table structure. @@ -2314,8 +2314,8 @@ clickhouse client \ --secure \ --password MY_PASSWORD \ --query " - insert into regexp_dictionary_source_table - select * from input ('id UInt64, parent_id UInt64, regexp String, keys Array(String), values Array(String)') + INSERT INTO regexp_dictionary_source_table + SELECT * FROM input ('id UInt64, parent_id UInt64, regexp String, keys Array(String), values Array(String)') FORMAT CSV" < regexp_dict.csv ``` @@ -2333,10 +2333,6 @@ LIFETIME(0) LAYOUT(regexp_tree); ``` -### Use RegexpTree Dictionary as a UA Parser - -With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) - ## Embedded Dictionaries {#embedded-dictionaries} diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference index 86a74291b07..4e72cf4ce37 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference @@ -1,11 +1,11 @@ 1 0 Linux/(\\d+[\\.\\d]*).+tlinux ['version','name'] ['\\1','TencentOS'] -2 0 (\\d+)/tclwebkit(\\d+[\\.\\d]*) ['comment','version','name'] ['test $1 and $2','$1','Andriod'] +2 0 (\\d+)/tclwebkit(\\d+[\\.\\d]*) ['comment','version','name'] ['test $1 and $2','$1','Android'] 3 2 33/tclwebkit ['version'] ['13'] 4 2 3[12]/tclwebkit ['version'] ['12'] 5 2 3[12]/tclwebkit ['version'] ['11'] 6 2 3[12]/tclwebkit ['version'] ['10'] ('TencentOS',101,'nothing') -('Andriod',13,'test 33 and 11.10') +('Android',13,'test 33 and 11.10') ('',NULL,'nothing') ('',0,'default') 30/tclwebkit0 @@ -23,22 +23,22 @@ 42/tclwebkit12 43/tclwebkit13 44/tclwebkit14 -('Andriod',30) -('Andriod',12) -('Andriod',12) -('Andriod',13) -('Andriod',34) -('Andriod',35) -('Andriod',36) -('Andriod',37) -('Andriod',38) -('Andriod',39) -('Andriod',40) -('Andriod',41) -('Andriod',42) -('Andriod',43) -('Andriod',44) -('Andriod1',33,'matched 3') -1 0 (\\d+)/tclwebkit ['version','name'] ['$1','Andriod'] +('Android',30) +('Android',12) +('Android',12) +('Android',13) +('Android',34) +('Android',35) +('Android',36) +('Android',37) +('Android',38) +('Android',39) +('Android',40) +('Android',41) +('Android',42) +('Android',43) +('Android',44) +('Android1',33,'matched 3') +1 0 (\\d+)/tclwebkit ['version','name'] ['$1','Android'] 2 0 33/tclwebkit ['comment','version'] ['matched 3','13'] -3 1 33/tclwebkit ['name'] ['Andriod1'] +3 1 33/tclwebkit ['name'] ['Android1'] diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql index 15e8adce403..42d7acbf057 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql @@ -1,7 +1,7 @@ -- Tags: use-vectorscan -DROP TABLE IF EXISTS regexp_dictionary_source_table; DROP DICTIONARY IF EXISTS regexp_dict1; +DROP TABLE IF EXISTS regexp_dictionary_source_table; CREATE TABLE regexp_dictionary_source_table ( @@ -15,7 +15,7 @@ CREATE TABLE regexp_dictionary_source_table -- test back reference. INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']) -INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Andriod', '$1', 'test $1 and $2']) +INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']) INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']) INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']) @@ -65,14 +65,14 @@ SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 } truncate table regexp_dictionary_source_table; INSERT INTO regexp_dictionary_source_table VALUES (1, 2, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']) -INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Andriod', '$1', 'test $1 and $2']) -INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Andriod', '$1', 'test $1 and $2']) +INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) +INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 } -- test priority truncate table regexp_dictionary_source_table; -INSERT INTO regexp_dictionary_source_table VALUES (1, 0, '(\d+)/tclwebkit', ['name', 'version'], ['Andriod', '$1']); -INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '33/tclwebkit', ['name'], ['Andriod1']); -- child has more priority than parents. +INSERT INTO regexp_dictionary_source_table VALUES (1, 0, '(\d+)/tclwebkit', ['name', 'version'], ['Android', '$1']); +INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '33/tclwebkit', ['name'], ['Android1']); -- child has more priority than parents. INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '33/tclwebkit', ['version', 'comment'], ['13', 'matched 3']); -- larger id has lower priority than small id. SYSTEM RELOAD dictionary regexp_dict1; select dictGet(regexp_dict1, ('name', 'version', 'comment'), '33/tclwebkit'); @@ -82,6 +82,6 @@ SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 } select * from dictionary(regexp_dict1); +DROP DICTIONARY IF EXISTS regexp_dict1; DROP TABLE IF EXISTS regexp_dictionary_source_table; DROP TABLE IF EXISTS needle_table; -DROP DICTIONARY IF EXISTS regexp_dict1; From 94fe22493527634853febfb848260c084c17a368 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 18 May 2023 10:06:59 -0300 Subject: [PATCH 048/129] Update partition.md --- docs/en/sql-reference/statements/alter/partition.md | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index ce0bc1ea528..a0aa74e6d25 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -284,13 +284,17 @@ Manipulates data in the specifies partition matching the specified filtering exp Syntax: ``` sql -ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_expr] WHERE filter_expr ``` ### Example ``` sql +-- using partition name ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2; + +-- using partition id +ALTER TABLE mt UPDATE x = x + 1 IN PARTITION ID '2' WHERE p = 2; ``` ### See Also @@ -304,13 +308,17 @@ Deletes data in the specifies partition matching the specified filtering express Syntax: ``` sql -ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE [IN PARTITION partition_expr] WHERE filter_expr ``` ### Example ``` sql +-- using partition name ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; + +-- using partition id +ALTER TABLE mt DELETE IN PARTITION ID '2' WHERE p = 2; ``` ### See Also From 6b4dcbd3ed5f3f00322b86cf82780509f93ea038 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 18 May 2023 23:23:39 +0800 Subject: [PATCH 049/129] Use PROJECT_*_DIR instead of CMAKE_*_DIR. --- CMakeLists.txt | 8 ++++---- cmake/add_check.cmake | 4 ++-- cmake/git.cmake | 10 +++++----- cmake/print_flags.cmake | 6 +++--- cmake/sanitize.cmake | 6 +++--- cmake/version.cmake | 2 +- contrib/avro-cmake/CMakeLists.txt | 2 +- contrib/cassandra-cmake/CMakeLists.txt | 2 +- contrib/cctz-cmake/CMakeLists.txt | 2 +- contrib/libuv-cmake/CMakeLists.txt | 4 ++-- contrib/mariadb-connector-c-cmake/CMakeLists.txt | 2 +- contrib/snappy-cmake/CMakeLists.txt | 2 +- contrib/zlib-ng-cmake/CMakeLists.txt | 2 +- programs/self-extracting/CMakeLists.txt | 6 +++--- src/CMakeLists.txt | 2 +- 15 files changed, 30 insertions(+), 30 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 26188cb7110..56bf3e1c3f8 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -259,8 +259,8 @@ endif () option (ENABLE_BUILD_PATH_MAPPING "Enable remapping of file source paths in debug info, predefined preprocessor macros, and __builtin_FILE(). It's used to generate reproducible builds. See https://reproducible-builds.org/docs/build-path" ${ENABLE_BUILD_PATH_MAPPING_DEFAULT}) if (ENABLE_BUILD_PATH_MAPPING) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.") - set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffile-prefix-map=${PROJECT_SOURCE_DIR}=.") + set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -ffile-prefix-map=${PROJECT_SOURCE_DIR}=.") endif () option (ENABLE_BUILD_PROFILING "Enable profiling of build time" OFF) @@ -557,7 +557,7 @@ if (NATIVE_BUILD_TARGETS ) message (STATUS "Building native targets...") - set (NATIVE_BUILD_DIR "${CMAKE_BINARY_DIR}/native") + set (NATIVE_BUILD_DIR "${PROJECT_BINARY_DIR}/native") execute_process( COMMAND ${CMAKE_COMMAND} -E make_directory "${NATIVE_BUILD_DIR}" @@ -571,7 +571,7 @@ if (NATIVE_BUILD_TARGETS # Avoid overriding .cargo/config.toml with native toolchain. "-DENABLE_RUST=OFF" "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=${ENABLE_CLICKHOUSE_SELF_EXTRACTING}" - ${CMAKE_SOURCE_DIR} + ${PROJECT_SOURCE_DIR} WORKING_DIRECTORY "${NATIVE_BUILD_DIR}" COMMAND_ECHO STDOUT) diff --git a/cmake/add_check.cmake b/cmake/add_check.cmake index c6abbcdb321..ba30ee8676f 100644 --- a/cmake/add_check.cmake +++ b/cmake/add_check.cmake @@ -5,11 +5,11 @@ if (NOT TARGET check) if (CMAKE_CONFIGURATION_TYPES) add_custom_target (check COMMAND ${CMAKE_CTEST_COMMAND} --force-new-ctest-process --output-on-failure --build-config "$" - WORKING_DIRECTORY ${CMAKE_BINARY_DIR}) + WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) else () add_custom_target (check COMMAND ${CMAKE_CTEST_COMMAND} --force-new-ctest-process --output-on-failure - WORKING_DIRECTORY ${CMAKE_BINARY_DIR}) + WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) endif () endif () diff --git a/cmake/git.cmake b/cmake/git.cmake index 397ec3cd081..a4b3bd4bdab 100644 --- a/cmake/git.cmake +++ b/cmake/git.cmake @@ -5,14 +5,14 @@ if (Git_FOUND) # Commit hash + whether the building workspace was dirty or not execute_process(COMMAND "${GIT_EXECUTABLE}" rev-parse HEAD - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_HASH ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) # Branch name execute_process(COMMAND "${GIT_EXECUTABLE}" rev-parse --abbrev-ref HEAD - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_BRANCH ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) @@ -20,14 +20,14 @@ if (Git_FOUND) SET(ENV{TZ} "UTC") execute_process(COMMAND "${GIT_EXECUTABLE}" log -1 --format=%ad --date=iso-local - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_DATE ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) # Subject of the commit execute_process(COMMAND "${GIT_EXECUTABLE}" log -1 --format=%s - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_COMMIT_SUBJECT ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) @@ -35,7 +35,7 @@ if (Git_FOUND) execute_process( COMMAND ${GIT_EXECUTABLE} status - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} OUTPUT_STRIP_TRAILING_WHITESPACE) + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_STRIP_TRAILING_WHITESPACE) else() message(STATUS "Git could not be found.") endif() diff --git a/cmake/print_flags.cmake b/cmake/print_flags.cmake index 66f2a8bfbc7..869764602d4 100644 --- a/cmake/print_flags.cmake +++ b/cmake/print_flags.cmake @@ -7,6 +7,6 @@ message (STATUS "compiler CXX = ${CMAKE_CXX_COMPILER} ${FULL_CXX_FLAGS}") message (STATUS "LINKER_FLAGS = ${FULL_EXE_LINKER_FLAGS}") # Reproducible builds -string (REPLACE "${CMAKE_SOURCE_DIR}" "." FULL_C_FLAGS_NORMALIZED "${FULL_C_FLAGS}") -string (REPLACE "${CMAKE_SOURCE_DIR}" "." FULL_CXX_FLAGS_NORMALIZED "${FULL_CXX_FLAGS}") -string (REPLACE "${CMAKE_SOURCE_DIR}" "." FULL_EXE_LINKER_FLAGS_NORMALIZED "${FULL_EXE_LINKER_FLAGS}") +string (REPLACE "${PROJECT_SOURCE_DIR}" "." FULL_C_FLAGS_NORMALIZED "${FULL_C_FLAGS}") +string (REPLACE "${PROJECT_SOURCE_DIR}" "." FULL_CXX_FLAGS_NORMALIZED "${FULL_CXX_FLAGS}") +string (REPLACE "${PROJECT_SOURCE_DIR}" "." FULL_EXE_LINKER_FLAGS_NORMALIZED "${FULL_EXE_LINKER_FLAGS}") diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index b2fbdb256fd..17ce8a7db29 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -29,14 +29,14 @@ if (SANITIZE) # Linking can fail due to relocation overflows (see #49145), caused by too big object files / libraries. # Work around this with position-independent builds (-fPIC and -fpie), this is slightly slower than non-PIC/PIE but that's okay. - set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/msan_suppressions.txt") + set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie -fsanitize-blacklist=${PROJECT_SOURCE_DIR}/tests/msan_suppressions.txt") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") elseif (SANITIZE STREQUAL "thread") set (TSAN_FLAGS "-fsanitize=thread") if (COMPILER_CLANG) - set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${PROJECT_SOURCE_DIR}/tests/tsan_suppressions.txt") endif() set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") @@ -54,7 +54,7 @@ if (SANITIZE) set(UBSAN_FLAGS "${UBSAN_FLAGS} -fno-sanitize=unsigned-integer-overflow") endif() if (COMPILER_CLANG) - set (UBSAN_FLAGS "${UBSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/ubsan_suppressions.txt") + set (UBSAN_FLAGS "${UBSAN_FLAGS} -fsanitize-blacklist=${PROJECT_SOURCE_DIR}/tests/ubsan_suppressions.txt") endif() set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}") diff --git a/cmake/version.cmake b/cmake/version.cmake index acaa772ff2f..9ca21556f4d 100644 --- a/cmake/version.cmake +++ b/cmake/version.cmake @@ -1,4 +1,4 @@ -include(${CMAKE_SOURCE_DIR}/cmake/autogenerated_versions.txt) +include(${PROJECT_SOURCE_DIR}/cmake/autogenerated_versions.txt) set(VERSION_EXTRA "" CACHE STRING "") set(VERSION_TWEAK "" CACHE STRING "") diff --git a/contrib/avro-cmake/CMakeLists.txt b/contrib/avro-cmake/CMakeLists.txt index 25474650d0e..63b3854eef9 100644 --- a/contrib/avro-cmake/CMakeLists.txt +++ b/contrib/avro-cmake/CMakeLists.txt @@ -6,7 +6,7 @@ if (NOT ENABLE_AVRO) return() endif() -set(AVROCPP_ROOT_DIR "${CMAKE_SOURCE_DIR}/contrib/avro/lang/c++") +set(AVROCPP_ROOT_DIR "${PROJECT_SOURCE_DIR}/contrib/avro/lang/c++") set(AVROCPP_INCLUDE_DIR "${AVROCPP_ROOT_DIR}/api") set(AVROCPP_SOURCE_DIR "${AVROCPP_ROOT_DIR}/impl") diff --git a/contrib/cassandra-cmake/CMakeLists.txt b/contrib/cassandra-cmake/CMakeLists.txt index 59ff908b63a..32611e0e151 100644 --- a/contrib/cassandra-cmake/CMakeLists.txt +++ b/contrib/cassandra-cmake/CMakeLists.txt @@ -18,7 +18,7 @@ endif() # Need to use C++17 since the compilation is not possible with C++20 currently. set (CMAKE_CXX_STANDARD 17) -set(CASS_ROOT_DIR ${CMAKE_SOURCE_DIR}/contrib/cassandra) +set(CASS_ROOT_DIR ${PROJECT_SOURCE_DIR}/contrib/cassandra) set(CASS_SRC_DIR "${CASS_ROOT_DIR}/src") set(CASS_INCLUDE_DIR "${CASS_ROOT_DIR}/include") diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index f1ef9b53f7d..10070fbd949 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -26,7 +26,7 @@ endif () # StorageSystemTimeZones.generated.cpp is autogenerated each time during a build # data in this file will be used to populate the system.time_zones table, this is specific to OS_LINUX # as the library that's built using embedded tzdata is also specific to OS_LINUX -set(SYSTEM_STORAGE_TZ_FILE "${CMAKE_BINARY_DIR}/src/Storages/System/StorageSystemTimeZones.generated.cpp") +set(SYSTEM_STORAGE_TZ_FILE "${PROJECT_BINARY_DIR}/src/Storages/System/StorageSystemTimeZones.generated.cpp") # remove existing copies so that its generated fresh on each build. file(REMOVE ${SYSTEM_STORAGE_TZ_FILE}) diff --git a/contrib/libuv-cmake/CMakeLists.txt b/contrib/libuv-cmake/CMakeLists.txt index fb88799ed38..928fdcdd7e6 100644 --- a/contrib/libuv-cmake/CMakeLists.txt +++ b/contrib/libuv-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ # This file is a modified version of contrib/libuv/CMakeLists.txt -set (SOURCE_DIR "${CMAKE_SOURCE_DIR}/contrib/libuv") -set (BINARY_DIR "${CMAKE_BINARY_DIR}/contrib/libuv") +set (SOURCE_DIR "${PROJECT_SOURCE_DIR}/contrib/libuv") +set (BINARY_DIR "${PROJECT_BINARY_DIR}/contrib/libuv") set(uv_sources src/fs-poll.c diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt index 50287c54ac1..18d1510a57b 100644 --- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt +++ b/contrib/mariadb-connector-c-cmake/CMakeLists.txt @@ -15,7 +15,7 @@ endif() # This is the LGPL libmariadb project. -set(CC_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/mariadb-connector-c) +set(CC_SOURCE_DIR ${PROJECT_SOURCE_DIR}/contrib/mariadb-connector-c) set(CC_BINARY_DIR ${CMAKE_CURRENT_BINARY_DIR}) set(WITH_SSL ON) diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt index 50cdc8732a1..f406de0e343 100644 --- a/contrib/snappy-cmake/CMakeLists.txt +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -set (SOURCE_DIR "${CMAKE_SOURCE_DIR}/contrib/snappy") +set (SOURCE_DIR "${PROJECT_SOURCE_DIR}/contrib/snappy") if (ARCH_S390X) set (SNAPPY_IS_BIG_ENDIAN 1) diff --git a/contrib/zlib-ng-cmake/CMakeLists.txt b/contrib/zlib-ng-cmake/CMakeLists.txt index aa067ba37e0..79f343bfc75 100644 --- a/contrib/zlib-ng-cmake/CMakeLists.txt +++ b/contrib/zlib-ng-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -set (SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/zlib-ng) +set (SOURCE_DIR ${PROJECT_SOURCE_DIR}/contrib/zlib-ng) add_definitions(-DZLIB_COMPAT) add_definitions(-DWITH_GZFILEOP) diff --git a/programs/self-extracting/CMakeLists.txt b/programs/self-extracting/CMakeLists.txt index 2cc26926b38..f3ff0bbcd78 100644 --- a/programs/self-extracting/CMakeLists.txt +++ b/programs/self-extracting/CMakeLists.txt @@ -4,10 +4,10 @@ if (NOT( AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR ) ) - set (COMPRESSOR "${CMAKE_BINARY_DIR}/native/utils/self-extracting-executable/pre_compressor") - set (DECOMPRESSOR "--decompressor=${CMAKE_BINARY_DIR}/utils/self-extracting-executable/decompressor") + set (COMPRESSOR "${PROJECT_BINARY_DIR}/native/utils/self-extracting-executable/pre_compressor") + set (DECOMPRESSOR "--decompressor=${PROJECT_BINARY_DIR}/utils/self-extracting-executable/decompressor") else () - set (COMPRESSOR "${CMAKE_BINARY_DIR}/utils/self-extracting-executable/compressor") + set (COMPRESSOR "${PROJECT_BINARY_DIR}/utils/self-extracting-executable/compressor") endif () add_custom_target (self-extracting ALL diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b3f4fbb7420..87a2979ecd1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -527,7 +527,7 @@ target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::fast_float) if (USE_ORC) dbms_target_link_libraries(PUBLIC ${ORC_LIBRARIES}) - dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} "${CMAKE_BINARY_DIR}/contrib/orc/c++/include") + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} "${PROJECT_BINARY_DIR}/contrib/orc/c++/include") endif () if (TARGET ch_contrib::rocksdb) From 8dc59c1efea2dfaa56ec5a0728a362714e2ab206 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 18 May 2023 21:40:20 +0000 Subject: [PATCH 050/129] Fix test_insert_same_partition_and_merge failing if one Azure request attempt fails --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- tests/integration/test_merge_tree_azure_blob_storage/test.py | 4 ++-- tests/integration/test_merge_tree_s3/test.py | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index f8ca6b9ab07..44185f74f60 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -51,7 +51,7 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, if (i == num_tries - 1) throw; - LOG_DEBUG(log, "Write at attempt {} for blob `{}` failed: {}", i + 1, blob_path, e.Message); + LOG_DEBUG(log, "Write at attempt {} for blob `{}` failed: {} {}", i + 1, blob_path, e.what(), e.Message); }; for (size_t i = 0; i < num_tries; ++i) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index bcb62c3181d..8bf4df17c39 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -203,7 +203,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): node.query(f"SYSTEM START MERGES {TABLE_NAME}") # Wait for merges and old parts deletion - for attempt in range(0, 10): + for attempt in range(0, 60): parts_count = azure_query( node, f"SELECT COUNT(*) FROM system.parts WHERE table = '{TABLE_NAME}' FORMAT Values", @@ -211,7 +211,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): if parts_count == "(1)": break - if attempt == 9: + if attempt == 59: assert parts_count == "(1)" time.sleep(1) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 7fbe8c8e99b..ee774f6632b 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -232,7 +232,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name): node.query("SYSTEM START MERGES s3_test") # Wait for merges and old parts deletion - for attempt in range(0, 10): + for attempt in range(0, 60): parts_count = node.query( "SELECT COUNT(*) FROM system.parts WHERE table = 's3_test' and active = 1 FORMAT Values" ) @@ -240,7 +240,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name): if parts_count == "(1)": break - if attempt == 9: + if attempt == 59: assert parts_count == "(1)" time.sleep(1) From c4f23e87f10be8f48c99be671a272347715676f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 18:56:56 +0200 Subject: [PATCH 051/129] Export grower_type in HashTable Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 8542ddae144..439b96d6976 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -753,6 +753,7 @@ protected: public: using key_type = Key; + using grower_type = Grower; using mapped_type = typename Cell::mapped_type; using value_type = typename Cell::value_type; using cell_type = Cell; From b44497fd4c1908b656bdd22538595084be63c9fe Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 16:33:46 +0200 Subject: [PATCH 052/129] Introduce PackedHashMap (HashMap with structure without padding) In case of you have HashMap with as the overhead of 38% can be crutial, especially if you have tons of keys. Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashMap.h | 18 +++++----- src/Common/HashTable/PackedHashMap.h | 46 ++++++++++++++++++++++++++ src/Interpreters/examples/hash_map.cpp | 31 +++++++++++++++++ 3 files changed, 87 insertions(+), 8 deletions(-) create mode 100644 src/Common/HashTable/PackedHashMap.h diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index f0045306fe8..5f4cb396822 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -9,6 +9,8 @@ /** NOTE HashMap could only be used for memmoveable (position independent) types. * Example: std::string is not position independent in libstdc++ with C++11 ABI or in libc++. * Also, key in hash table must be of type, that zero bytes is compared equals to zero key. + * + * Please keep in sync with PackedHashMap.h */ namespace DB @@ -53,13 +55,13 @@ PairNoInit, std::decay_t> makePairNoInit(First && fi } -template +template > struct HashMapCell { using Mapped = TMapped; using State = TState; - using value_type = PairNoInit; + using value_type = Pair; using mapped_type = Mapped; using key_type = Key; @@ -151,14 +153,14 @@ struct HashMapCell namespace std { - template - struct tuple_size> : std::integral_constant { }; + template + struct tuple_size> : std::integral_constant { }; - template - struct tuple_element<0, HashMapCell> { using type = Key; }; + template + struct tuple_element<0, HashMapCell> { using type = Key; }; - template - struct tuple_element<1, HashMapCell> { using type = TMapped; }; + template + struct tuple_element<1, HashMapCell> { using type = TMapped; }; } template diff --git a/src/Common/HashTable/PackedHashMap.h b/src/Common/HashTable/PackedHashMap.h new file mode 100644 index 00000000000..6b52a04aa49 --- /dev/null +++ b/src/Common/HashTable/PackedHashMap.h @@ -0,0 +1,46 @@ +#pragma once + +/// Packed versions HashMap, please keep in sync with HashMap.h + +#include + +/// A pair that does not initialize the elements, if not needed. +/// +/// NOTE: makePairNoInit() is omitted for PackedPairNoInit since it is not +/// required for PackedHashMap (see mergeBlockWithPipe() for details) +template +struct PackedPairNoInit +{ + First first; + Second second; + + PackedPairNoInit() {} /// NOLINT + + template + PackedPairNoInit(FirstValue && first_, NoInitTag) + : first(std::forward(first_)) + { + } + + template + PackedPairNoInit(FirstValue && first_, SecondValue && second_) + : first(std::forward(first_)) + , second(std::forward(second_)) + { + } +} __attribute__((packed)); + +/// Packed HashMap - HashMap with structure without padding +/// +/// Sometimes padding in structure can be crucial, consider the following +/// example as in this case the padding overhead +/// is 0.375, and this can be major in case of lots of keys. +/// +/// Note, there is no need to provide PackedHashSet, since it cannot have padding. +template < + typename Key, + typename Mapped, + typename Hash = DefaultHash, + typename Grower = HashTableGrower<>, + typename Allocator = HashTableAllocator> +using PackedHashMap = HashMapTable>, Hash, Grower, Allocator>; diff --git a/src/Interpreters/examples/hash_map.cpp b/src/Interpreters/examples/hash_map.cpp index fe2cbda019d..c0016bf8cd8 100644 --- a/src/Interpreters/examples/hash_map.cpp +++ b/src/Interpreters/examples/hash_map.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -278,5 +279,35 @@ int main(int argc, char ** argv) << std::endl; } + if (argc < 3 || std::stol(argv[2]) == 7) + { + Stopwatch watch; + + PackedHashMap map; + PackedHashMap::LookupResult it; + bool inserted; + + for (size_t i = 0; i < n; ++i) + { + map.emplace(data[i], it, inserted); + if (inserted) + { + new (&it->getMapped()) Value; + std::swap(it->getMapped(), value); + INIT + } + } + + watch.stop(); + std::cerr << std::fixed << std::setprecision(2) + << "PackedHashMap. Size: " << map.size() + << ", elapsed: " << watch.elapsedSeconds() + << " (" << n / watch.elapsedSeconds() << " elem/sec.)" +#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS + << ", collisions: " << map.getCollisions() +#endif + << std::endl; + } + return 0; } From 7b5d156cc5e4d53091a1d2930fe6def0fafe8b5e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 18:01:01 +0200 Subject: [PATCH 053/129] Optimize SPARSE_HASHED layout (by using PackedHashMap) In case you want dictionary optimized for memory, SPARSE_HASHED is not always gives you what you need. Consider the following example as , but this pair will also have a 6 byte padding (on amd64), so this is almost 40% of space wastage. And because of this padding, even google::sparse_hash_map, does not make picture better, in fact, sparse_hash_map is not very friendly to memory allocators (especially jemalloc). Here are some numbers for dictionary with 1e9 elements and UInt64 as key, and UInt16 as value: settings | load (sec) | read (sec) | read (million rows/s) | bytes_allocated | RSS HASHED upstream | - | - | - | - | 35GiB SPARSE_HASHED upstream | - | - | - | - | 26GiB - | - | - | - | - | - sparse_hash_map glibc hashbench | - | - | - | - | 17.5GiB sparse_hash_map packed allocator | 101.878 | 231.48 | 4.32 | - | 17.7GiB PackedHashMap | 15.514 | 42.35 | 23.61 | 20GiB | 22GiB As you can see PackedHashMap looks way more better then HASHED, and even better then SPARSE_HASHED, but slightly worse then sparse_hash_map with packed allocator (it is done with a custom patch to google sparse_hash_map). v2: rebase on top of bucket_count fix Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 74 ++-------- src/Dictionaries/HashedDictionary.h | 44 +----- .../HashedDictionaryCollectionTraits.h | 102 +++++++++++++ .../HashedDictionaryCollectionType.h | 134 ++++++++++++++++++ .../performance/hashed_dictionary_sharded.xml | 2 +- 5 files changed, 249 insertions(+), 107 deletions(-) create mode 100644 src/Dictionaries/HashedDictionaryCollectionTraits.h create mode 100644 src/Dictionaries/HashedDictionaryCollectionType.h diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 191adab8983..048c378b917 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -28,21 +29,6 @@ namespace CurrentMetrics extern const Metric HashedDictionaryThreadsActive; } -namespace -{ - -/// NOTE: Trailing return type is explicitly specified for SFINAE. - -/// google::sparse_hash_map -template auto getKeyFromCell(const T & value) -> decltype(value->first) { return value->first; } // NOLINT -template auto getValueFromCell(const T & value) -> decltype(value->second) { return value->second; } // NOLINT - -/// HashMap -template auto getKeyFromCell(const T & value) -> decltype(value->getKey()) { return value->getKey(); } // NOLINT -template auto getValueFromCell(const T & value) -> decltype(value->getMapped()) { return value->getMapped(); } // NOLINT - -} - namespace DB { @@ -242,10 +228,7 @@ HashedDictionary::~HashedDictionary() CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictDtor"); - if constexpr (sparse) - container.clear(); - else - container.clearAndShrink(); + clearContainer(container); }); ++hash_tables_count; @@ -834,12 +817,7 @@ void HashedDictionary::resize(size_t added if (unlikely(attributes_size == 0)) { size_t reserve_size = added_rows + no_attributes_containers.front().size(); - - if constexpr (sparse) - no_attributes_containers.front().resize(reserve_size); - else - no_attributes_containers.front().reserve(reserve_size); - + resizeContainer(no_attributes_containers.front(), reserve_size); return; } @@ -849,11 +827,7 @@ void HashedDictionary::resize(size_t added { auto & container = containers.front(); size_t reserve_size = added_rows + container.size(); - - if constexpr (sparse) - container.resize(reserve_size); - else - container.reserve(reserve_size); + resizeContainer(container, reserve_size); }); } } @@ -973,25 +947,9 @@ void HashedDictionary::calculateBytesAlloc { for (const auto & container : containers) { - using ContainerType = std::decay_t; - using AttributeValueType = typename ContainerType::mapped_type; - bytes_allocated += sizeof(container); - - if constexpr (sparse || std::is_same_v) - { - /// bucket_count() - Returns table size, that includes empty and deleted - /// size() - Returns table size, without empty and deleted - /// and since this is sparsehash, empty cells should not be significant, - /// and since items cannot be removed from the dictionary, deleted is also not important. - bytes_allocated += container.size() * (sizeof(KeyType) + sizeof(AttributeValueType)); - bucket_count += container.bucket_count(); - } - else - { - bytes_allocated += container.getBufferSizeInBytes(); - bucket_count += container.getBufferSizeInCells(); - } + bytes_allocated += getBufferSizeInBytes(container); + bucket_count += getBufferSizeInCells(container); } }); @@ -1010,17 +968,8 @@ void HashedDictionary::calculateBytesAlloc for (const auto & container : no_attributes_containers) { bytes_allocated += sizeof(container); - - if constexpr (sparse) - { - bytes_allocated += container.size() * (sizeof(KeyType)); - bucket_count += container.bucket_count(); - } - else - { - bytes_allocated += container.getBufferSizeInBytes(); - bucket_count += container.getBufferSizeInCells(); - } + bytes_allocated += getBufferSizeInBytes(container); + bucket_count += getBufferSizeInCells(container); } } @@ -1078,12 +1027,7 @@ Pipe HashedDictionary::read(const Names & keys.reserve(keys.size() + container.size()); for (const auto & key : container) - { - if constexpr (sparse) - keys.emplace_back(key); - else - keys.emplace_back(key.getKey()); - } + keys.emplace_back(getSetKeyFromCell(key)); } } diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 67683679647..08f5dfaa40d 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -4,17 +4,14 @@ #include #include #include -#include -#include -#include -#include #include #include #include #include #include +#include /** This dictionary stores all content in a hash table in memory * (a separate Key -> Value map for each attribute) @@ -136,42 +133,7 @@ public: private: template - using CollectionTypeNonSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::Simple, - HashMap>, - HashMapWithSavedHash>>; - - using NoAttributesCollectionTypeNonSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::Simple, - HashSet>, - HashSetWithSavedHash>>; - - /// Here we use sparse_hash_map with DefaultHash<> for the following reasons: - /// - /// - DefaultHash<> is used for HashMap - /// - DefaultHash<> (from HashTable/Hash.h> works better then std::hash<> - /// in case of sequential set of keys, but with random access to this set, i.e. - /// - /// SELECT number FROM numbers(3000000) ORDER BY rand() - /// - /// And even though std::hash<> works better in some other cases, - /// DefaultHash<> is preferred since the difference for this particular - /// case is significant, i.e. it can be 10x+. - template - using CollectionTypeSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::Simple, - google::sparse_hash_map>, - google::sparse_hash_map>>; - - using NoAttributesCollectionTypeSparse = google::sparse_hash_set>; - - template - using CollectionType = std::conditional_t, CollectionTypeNonSparse>; - - template - using CollectionsHolder = std::vector>; - - using NoAttributesCollectionType = std::conditional_t; + using CollectionsHolder = std::vector::Type>; using NullableSet = HashSet>; using NullableSets = std::vector; @@ -269,7 +231,7 @@ private: BlockPtr update_field_loaded_block; std::vector> string_arenas; - std::vector no_attributes_containers; + std::vector::Type> no_attributes_containers; DictionaryHierarchicalParentToChildIndexPtr hierarchical_index; }; diff --git a/src/Dictionaries/HashedDictionaryCollectionTraits.h b/src/Dictionaries/HashedDictionaryCollectionTraits.h new file mode 100644 index 00000000000..20fc882a2fe --- /dev/null +++ b/src/Dictionaries/HashedDictionaryCollectionTraits.h @@ -0,0 +1,102 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// sparse_hash_map/sparse_hash_set +template +concept IsGoogleSparseHashTable = std::is_same_v>>; + +template +concept IsStdMapCell = requires (V v) +{ + v->first; + v->second; +}; + +/// HashMap/HashMapWithSavedHash/HashSet/HashMapWithSavedHash/PackedHashMap and their Cells +template +concept IsBuiltinHashTable = ( + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> +); + +template +concept IsBuiltinSetCell = requires (V v) +{ + v.getKey(); +}; + +template +concept IsBuiltinMapCell = requires (V v) +{ + v->getKey(); + v->getMapped(); +}; + +// NOLINTBEGIN(*) + +/// google::sparse_hash_map +template auto getSetKeyFromCell(const T & value) { return value; } +template auto getKeyFromCell(const T & value) requires (IsStdMapCell) { return value->first; } +template auto getValueFromCell(const T & value) requires (IsStdMapCell) { return value->second; } + +/// size() - returns table size, without empty and deleted +/// and since this is sparsehash, empty cells should not be significant, +/// and since items cannot be removed from the dictionary, deleted is also not important. +/// +/// NOTE: for google::sparse_hash_set value_type is Key, for sparse_hash_map +/// value_type is std::pair, and now we correctly takes into +/// account padding in structures, if any. +template auto getBufferSizeInBytes(const C & c) requires (IsGoogleSparseHashTable) { return c.size() * sizeof(typename C::value_type); } +/// bucket_count() - Returns table size, that includes empty and deleted +template auto getBufferSizeInCells(const C & c) requires (IsGoogleSparseHashTable) { return c.bucket_count(); } + +template auto resizeContainer(C & c, size_t size) requires (IsGoogleSparseHashTable) { return c.resize(size); } +template auto clearContainer(C & c) requires (IsGoogleSparseHashTable) { return c.clear(); } + +/// HashMap +template auto getSetKeyFromCell(const T & value) requires (IsBuiltinSetCell) { return value.getKey(); } +template auto getKeyFromCell(const T & value) requires (IsBuiltinMapCell) { return value->getKey(); } +template auto getValueFromCell(const T & value) requires (IsBuiltinMapCell) { return value->getMapped(); } + +template auto getBufferSizeInBytes(const C & c) requires (IsBuiltinHashTable) { return c.getBufferSizeInBytes(); } +template auto getBufferSizeInCells(const C & c) requires (IsBuiltinHashTable) { return c.getBufferSizeInCells(); } +template auto resizeContainer(C & c, size_t size) requires (IsBuiltinHashTable) { return c.reserve(size); } +template void clearContainer(C & c) requires (IsBuiltinHashTable) { return c.clearAndShrink(); } + +// NOLINTEND(*) + +} diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h new file mode 100644 index 00000000000..05bdfbbf00f --- /dev/null +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -0,0 +1,134 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// HashMap with packed structure is better than google::sparse_hash_map if the +/// pair is small, for the sizeof(std::pair) == 16, RSS for hash +/// table with 1e9 elements will be: +/// +/// - google::sparse_hash_map : 26GiB +/// - HashMap : 35GiB +/// - PackedHashMap : 22GiB +/// - google::sparse_hash_map: 17GiB +/// +/// Also note here sizeof(std::pair<>) was used since google::sparse_hash_map +/// uses it to store , yes we can modify google::sparse_hash_map to work +/// with packed analog of std::pair, but the allocator overhead is still +/// significant, because of tons of reallocations (and those cannot be solved +/// with reserve() due to some internals of google::sparse_hash_map) and poor +/// jemalloc support of such pattern, which results in 33% fragmentation (in +/// comparison with glibc). +/// +/// Plus since google::sparse_hash_map cannot use packed structure, it will +/// have the same memory footprint for everything from UInt8 to UInt64 values +/// and so on. +/// +/// Returns true hen google::sparse_hash_map should be used, otherwise +/// PackedHashMap should be used instead. +template +constexpr bool useSparseHashForHashedDictionary() +{ + return sizeof(PackedPairNoInit) > 16; +} + +/// +/// Map (dictionary with attributes) +/// + +/// Type of the hash table for the dictionary. +template +struct HashedDictionaryMapType; + +/// Default implementation using builtin HashMap (for HASHED layout). +template +struct HashedDictionaryMapType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashMap>, + HashMapWithSavedHash>>; +}; + +/// Implementations for SPARSE_HASHED layout. +template +struct HashedDictionarySparseMapType; + +/// Implementation based on google::sparse_hash_map for SPARSE_HASHED. +template +struct HashedDictionarySparseMapType +{ + /// Here we use sparse_hash_map with DefaultHash<> for the following reasons: + /// + /// - DefaultHash<> is used for HashMap + /// - DefaultHash<> (from HashTable/Hash.h> works better then std::hash<> + /// in case of sequential set of keys, but with random access to this set, i.e. + /// + /// SELECT number FROM numbers(3000000) ORDER BY rand() + /// + /// And even though std::hash<> works better in some other cases, + /// DefaultHash<> is preferred since the difference for this particular + /// case is significant, i.e. it can be 10x+. + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + google::sparse_hash_map>, + google::sparse_hash_map>>; +}; + +/// Implementation based on PackedHashMap for SPARSE_HASHED. +template +struct HashedDictionarySparseMapType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + PackedHashMap>, + PackedHashMap>>; +}; +template +struct HashedDictionaryMapType + : public HashedDictionarySparseMapType< + dictionary_key_type, Key, Value, + /* use_sparse_hash= */ useSparseHashForHashedDictionary()> +{}; + +/// +/// Set (dictionary with attributes) +/// + +/// Type of the hash table for the dictionary. +template +struct HashedDictionarySetType; + +/// Default implementation using builtin HashMap (for HASHED layout). +template +struct HashedDictionarySetType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashSet>, + HashSetWithSavedHash>>; +}; + +/// Implementation for SPARSE_HASHED. +/// +/// NOTE: There is no implementation based on google::sparse_hash_set since +/// PackedHashMap is more optimal anyway (see comments for +/// useSparseHashForHashedDictionary()). +template +struct HashedDictionarySetType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashSet>, + HashSet>>; +}; + +} diff --git a/tests/performance/hashed_dictionary_sharded.xml b/tests/performance/hashed_dictionary_sharded.xml index 98c65fc49ea..d229ce02c49 100644 --- a/tests/performance/hashed_dictionary_sharded.xml +++ b/tests/performance/hashed_dictionary_sharded.xml @@ -22,7 +22,7 @@ CREATE TABLE simple_key_dictionary_source_table ( id UInt64, - value_int UInt64 + value_int UInt16 ) ENGINE = Memory From fb6f7631c20c4a1cc23b4743491804536d0b67ae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 20:21:04 +0200 Subject: [PATCH 054/129] Add ability to pass grower for HashTable during creation Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 439b96d6976..0e8510d1c8d 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -771,6 +771,14 @@ public: alloc(grower); } + explicit HashTable(const Grower & grower_) + : grower(grower_) + { + if (Cell::need_zero_value_storage) + this->zeroValue()->setZero(); + alloc(grower); + } + HashTable(size_t reserve_for_num_elements) /// NOLINT { if (Cell::need_zero_value_storage) From 8c6d691f52dc09958088f6b0b5f6820c4a70b47d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 21:06:18 +0200 Subject: [PATCH 055/129] Use HashTable constructor in HashSet Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashSet.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/HashTable/HashSet.h b/src/Common/HashTable/HashSet.h index 8f3761599ab..c25bfb14d9c 100644 --- a/src/Common/HashTable/HashSet.h +++ b/src/Common/HashTable/HashSet.h @@ -41,6 +41,8 @@ public: using Base = HashTable; using typename Base::LookupResult; + using Base::Base; + void merge(const Self & rhs) { if (!this->hasZero() && rhs.hasZero()) From 3698302ddb32b489aa7e3ad556318f8e0a39f0c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 21:28:20 +0200 Subject: [PATCH 056/129] Accept float values for dictionary layouts configurations Signed-off-by: Azat Khuzhin --- src/Dictionaries/getDictionaryConfigurationFromAST.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 05065df5251..0b7352e9cbb 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -156,11 +156,11 @@ void buildLayoutConfiguration( const auto value_field = value_literal->value; - if (value_field.getType() != Field::Types::UInt64 && value_field.getType() != Field::Types::String) + if (value_field.getType() != Field::Types::UInt64 && value_field.getType() != Field::Types::Float64 && value_field.getType() != Field::Types::String) { throw DB::Exception( ErrorCodes::BAD_ARGUMENTS, - "Dictionary layout parameter value must be an UInt64 or String, got '{}' instead", + "Dictionary layout parameter value must be an UInt64, Float64 or String, got '{}' instead", value_field.getTypeName()); } From 2996b3860612b6990867c197d630fc50528ec5b8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 20:34:47 +0200 Subject: [PATCH 057/129] Add ability to configure maximum load factor for the HASHED/SPARSE_HASHED layout As it turns out, HashMap/PackedHashMap works great even with max load factor of 0.99. By "great" I mean it least it works faster then google sparsehash, and not to mention it's friendliness to the memory allocator (it has zero fragmentation since it works with a continuious memory region, in comparison to the sparsehash that doing lots of realloc, which jemalloc does not like, due to it's slabs). Here is a table of different setups: settings | load (sec) | read (sec) | read (million rows/s) | bytes_allocated | RSS - | - | - | - | - | - HASHED upstream | - | - | - | - | 35GiB SPARSE_HASHED upstream | - | - | - | - | 26GiB - | - | - | - | - | - sparse_hash_map glibc hashbench | - | - | - | - | 17.5GiB sparse_hash_map packed allocator | 101.878 | 231.48 | 4.32 | - | 17.7GiB PackedHashMap 0.5 | 15.514 | 42.35 | 23.61 | 20GiB | 22GiB hashed 0.95 | 34.903 | 115.615 | 8.65 | 16GiB | 18.7GiB **PackedHashMap 0.95** | **93.6** | **19.883** | **10.68** | **10GiB** | **12.8GiB** PackedHashMap 0.99 | 26.113 | 83.6 | 11.96 | 10GiB | 12.3GiB As it shows, PackedHashMap with 0.95 max_load_factor, eats 2.6x less memory then SPARSE_HASHED in upstream, and it also 2x faster for read! v2: fix grower Signed-off-by: Azat Khuzhin --- docs/en/sql-reference/dictionaries/index.md | 8 ++ src/Dictionaries/HashedDictionary.cpp | 30 ++++- src/Dictionaries/HashedDictionary.h | 1 + .../HashedDictionaryCollectionType.h | 102 +++++++++++++++-- .../hashed_dictionary_load_factor.xml | 92 +++++++++++++++ ...hed_dictionary_load_factor_smoke.reference | 12 ++ ...40_hashed_dictionary_load_factor_smoke.sql | 107 ++++++++++++++++++ ...41_hashed_dictionary_load_factor.reference | 4 + ...02741_hashed_dictionary_load_factor.sql.j2 | 39 +++++++ 9 files changed, 384 insertions(+), 11 deletions(-) create mode 100644 tests/performance/hashed_dictionary_load_factor.xml create mode 100644 tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference create mode 100644 tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql create mode 100644 tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference create mode 100644 tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 5801b7866cb..dfda1ff9c04 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -284,6 +284,13 @@ Configuration example: 10000 is good balance between memory and speed. Even for 10e10 elements and can handle all the load without starvation. --> 10000 + + 0.5 ``` @@ -327,6 +334,7 @@ Configuration example: 1 + ``` diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 048c378b917..fa9a846195d 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -630,6 +630,8 @@ void HashedDictionary::createAttributes() const auto size = dict_struct.attributes.size(); attributes.reserve(size); + HashTableGrowerWithMaxLoadFactor grower(configuration.max_load_factor); + for (const auto & dictionary_attribute : dict_struct.attributes) { auto type_call = [&, this](const auto & dictionary_attribute_type) @@ -639,8 +641,23 @@ void HashedDictionary::createAttributes() using ValueType = DictionaryValueType; auto is_nullable_sets = dictionary_attribute.is_nullable ? std::make_optional(configuration.shards) : std::optional{}; - Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_sets), CollectionsHolder(configuration.shards)}; - attributes.emplace_back(std::move(attribute)); + if constexpr (IsBuiltinHashTable::value_type>) + { + CollectionsHolder collections; + collections.reserve(configuration.shards); + for (size_t i = 0; i < configuration.shards; ++i) + collections.emplace_back(grower); + + Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_sets), std::move(collections)}; + attributes.emplace_back(std::move(attribute)); + } + else + { + Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_sets), CollectionsHolder(configuration.shards)}; + for (auto & container : std::get>(attribute.containers)) + container.max_load_factor(configuration.max_load_factor); + attributes.emplace_back(std::move(attribute)); + } }; callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); @@ -648,7 +665,9 @@ void HashedDictionary::createAttributes() if (unlikely(attributes.size()) == 0) { - no_attributes_containers.resize(configuration.shards); + no_attributes_containers.reserve(configuration.shards); + for (size_t i = 0; i < configuration.shards; ++i) + no_attributes_containers.emplace_back(grower); } string_arenas.resize(configuration.shards); @@ -1136,9 +1155,14 @@ void registerDictionaryHashed(DictionaryFactory & factory) if (shard_load_queue_backlog <= 0) throw Exception(ErrorCodes::BAD_ARGUMENTS,"{}: SHARD_LOAD_QUEUE_BACKLOG parameter should be greater then zero", full_name); + float max_load_factor = static_cast(config.getDouble(config_prefix + dictionary_layout_prefix + ".max_load_factor", 0.5)); + if (max_load_factor < 0.5 || max_load_factor > 0.99) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: max_load_factor parameter should be within [0.5, 0.99], got {}", full_name, max_load_factor); + HashedDictionaryConfiguration configuration{ static_cast(shards), static_cast(shard_load_queue_backlog), + max_load_factor, require_nonempty, dict_lifetime, }; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 08f5dfaa40d..bff22706435 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -25,6 +25,7 @@ struct HashedDictionaryConfiguration { const UInt64 shards; const UInt64 shard_load_queue_backlog; + const float max_load_factor; const bool require_nonempty; const DictionaryLifetime lifetime; }; diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 05bdfbbf00f..1ab0ff8b0e3 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -40,6 +40,92 @@ constexpr bool useSparseHashForHashedDictionary() return sizeof(PackedPairNoInit) > 16; } +/// Grower with custom fill limit/load factor (instead of default 50%). +/// +/// It turns out that HashMap can outperform google::sparse_hash_map in case of +/// the structure size of not big, in terms of speed *and* memory. Even 99% of +/// max load factor was faster then google::sparse_hash_map in my simple tests +/// (1e9 UInt64 keys with UInt16 values, randomly distributed). +/// +/// And not to mention very high allocator memory fragmentation in +/// google::sparse_hash_map. +/// +/// Based on HashTableGrowerWithPrecalculation +class alignas(64) HashTableGrowerWithMaxLoadFactor +{ + static constexpr size_t initial_size_degree = 8; + UInt8 size_degree = initial_size_degree; + size_t precalculated_mask = (1ULL << initial_size_degree) - 1; + size_t precalculated_max_fill = 1ULL << (initial_size_degree - 1); + float max_load_factor = 0.5; + /// HashTableGrowerWithPrecalculation has 23, but to decrease memory usage + /// at least slightly 19 is used here. Also note, that for dictionaries it + /// is not that important since they are not that frequently loaded. + static constexpr size_t max_size_degree_quadratic = 19; + +public: + static constexpr auto initial_count = 1ULL << initial_size_degree; + + /// If collision resolution chains are contiguous, we can implement erase operation by moving the elements. + static constexpr auto performs_linear_probing_with_single_step = true; + + HashTableGrowerWithMaxLoadFactor() = default; + explicit HashTableGrowerWithMaxLoadFactor(float max_load_factor_) + : max_load_factor(max_load_factor_) + { + increaseSizeDegree(0); + } + + UInt8 sizeDegree() const { return size_degree; } + + void increaseSizeDegree(UInt8 delta) + { + size_degree += delta; + precalculated_mask = (1ULL << size_degree) - 1; + precalculated_max_fill = static_cast((1ULL << size_degree) * max_load_factor); + } + + /// The size of the hash table in the cells. + size_t bufSize() const { return 1ULL << size_degree; } + + /// From the hash value, get the cell number in the hash table. + size_t place(size_t x) const { return x & precalculated_mask; } + + /// The next cell in the collision resolution chain. + size_t next(size_t pos) const { return (pos + 1) & precalculated_mask; } + + /// Whether the hash table is sufficiently full. You need to increase the size of the hash table, or remove something unnecessary from it. + bool overflow(size_t elems) const { return elems > precalculated_max_fill; } + + /// Increase the size of the hash table. + void increaseSize() { increaseSizeDegree(size_degree >= max_size_degree_quadratic ? 1 : 2); } + + /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. + void set(size_t num_elems) + { + if (num_elems <= 1) + size_degree = initial_size_degree; + else if (initial_size_degree > static_cast(log2(num_elems - 1)) + 2) + size_degree = initial_size_degree; + else + { + /// Slightly more optimal than HashTableGrowerWithPrecalculation + /// and takes into account max_load_factor. + size_degree = static_cast(log2(num_elems - 1)) + 1; + if ((1ULL << size_degree) * max_load_factor < num_elems) + ++size_degree; + } + increaseSizeDegree(0); + } + + void setBufSize(size_t buf_size_) + { + size_degree = static_cast(log2(buf_size_ - 1) + 1); + increaseSizeDegree(0); + } +}; +static_assert(sizeof(HashTableGrowerWithMaxLoadFactor) == 64); + /// /// Map (dictionary with attributes) /// @@ -54,8 +140,8 @@ struct HashedDictionaryMapType>, - HashMapWithSavedHash>>; + HashMap, HashTableGrowerWithMaxLoadFactor>, + HashMapWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; }; /// Implementations for SPARSE_HASHED layout. @@ -89,8 +175,8 @@ struct HashedDictionarySparseMapType>, - PackedHashMap>>; + PackedHashMap, HashTableGrowerWithMaxLoadFactor>, + PackedHashMap, HashTableGrowerWithMaxLoadFactor>>; }; template struct HashedDictionaryMapType @@ -113,8 +199,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet>, - HashSetWithSavedHash>>; + HashSet, HashTableGrowerWithMaxLoadFactor>, + HashSetWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; }; /// Implementation for SPARSE_HASHED. @@ -127,8 +213,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet>, - HashSet>>; + HashSet, HashTableGrowerWithMaxLoadFactor>, + HashSet, HashTableGrowerWithMaxLoadFactor>>; }; } diff --git a/tests/performance/hashed_dictionary_load_factor.xml b/tests/performance/hashed_dictionary_load_factor.xml new file mode 100644 index 00000000000..dfc5c64b133 --- /dev/null +++ b/tests/performance/hashed_dictionary_load_factor.xml @@ -0,0 +1,92 @@ + + + + layout_suffix + + HASHED + SPARSE_HASHED + + + + + load_factor + + + 5 + 7 + 99 + + + + + + CREATE TABLE simple_key_dictionary_source_table + ( + id UInt64, + value_int UInt16 + ) ENGINE = Memory + + + + CREATE TABLE complex_key_dictionary_source_table + ( + id UInt64, + id_key String, + value_int UInt64 + ) ENGINE = Memory + + + + CREATE DICTIONARY IF NOT EXISTS simple_key_{layout_suffix}_dictionary_l0_{load_factor} + ( + id UInt64, + value_int UInt64 + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(TABLE 'simple_key_dictionary_source_table')) + LAYOUT({layout_suffix}(MAX_LOAD_FACTOR 0.{load_factor})) + LIFETIME(0) + + + + CREATE DICTIONARY IF NOT EXISTS complex_key_{layout_suffix}_dictionary_l0_{load_factor} + ( + id UInt64, + id_key String, + value_int UInt64 + ) + PRIMARY KEY id, id_key + SOURCE(CLICKHOUSE(TABLE 'complex_key_dictionary_source_table')) + LAYOUT(COMPLEX_KEY_{layout_suffix}(MAX_LOAD_FACTOR 0.{load_factor})) + LIFETIME(0) + + + INSERT INTO simple_key_dictionary_source_table SELECT number, number FROM numbers(3_000_000) + INSERT INTO complex_key_dictionary_source_table SELECT number, toString(number), number FROM numbers(2_000_000) + + SYSTEM RELOAD DICTIONARY simple_key_{layout_suffix}_dictionary_l0_{load_factor} + SYSTEM RELOAD DICTIONARY complex_key_{layout_suffix}_dictionary_l0_{load_factor} + + SYSTEM RELOAD DICTIONARY simple_key_{layout_suffix}_dictionary_l0_{load_factor} + SYSTEM RELOAD DICTIONARY complex_key_{layout_suffix}_dictionary_l0_{load_factor} + + + WITH rand64() % 3_000_000 as key + SELECT dictHas('default.simple_key_{layout_suffix}_dictionary_l0_{load_factor}', key) + FROM numbers(3_000_000) + FORMAT Null + + + + WITH (rand64() % 2_000_000, toString(rand64() % 2_000_000)) as key + SELECT dictHas('default.complex_key_{layout_suffix}_dictionary_l0_{load_factor}', key) + FROM numbers(2_000_000) + FORMAT Null + + + DROP DICTIONARY simple_key_{layout_suffix}_dictionary_l0_{load_factor} + DROP DICTIONARY complex_key_{layout_suffix}_dictionary_l0_{load_factor} + + DROP TABLE simple_key_dictionary_source_table + DROP TABLE complex_key_dictionary_source_table + diff --git a/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference new file mode 100644 index 00000000000..d43fa8e734c --- /dev/null +++ b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference @@ -0,0 +1,12 @@ +CREATE DICTIONARY default.test_sparse_dictionary_load_factor\n(\n `key` UInt64,\n `value` UInt16\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(TABLE test_table))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(SPARSE_HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 +CREATE DICTIONARY default.test_dictionary_load_factor\n(\n `key` UInt64,\n `value` UInt16\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(TABLE test_table))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 +CREATE DICTIONARY default.test_dictionary_load_factor_nullable\n(\n `key` UInt64,\n `value` Nullable(UInt16)\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(TABLE test_table_nullable))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 +CREATE DICTIONARY default.test_complex_dictionary_load_factor\n(\n `key_1` UInt64,\n `key_2` UInt64,\n `value` UInt16\n)\nPRIMARY KEY key_1, key_2\nSOURCE(CLICKHOUSE(TABLE test_table_complex))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(COMPLEX_KEY_HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 diff --git a/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql new file mode 100644 index 00000000000..d4bb9a1b14a --- /dev/null +++ b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql @@ -0,0 +1,107 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + key UInt64, + value UInt16 +) ENGINE=Memory() AS SELECT number, number FROM numbers(1e5); + +DROP TABLE IF EXISTS test_table_nullable; +CREATE TABLE test_table_nullable +( + key UInt64, + value Nullable(UInt16) +) ENGINE=Memory() AS SELECT number, number % 2 == 0 ? NULL : number FROM numbers(1e5); + +DROP TABLE IF EXISTS test_table_string; +CREATE TABLE test_table_string +( + key String, + value UInt16 +) ENGINE=Memory() AS SELECT 'foo' || number::String, number FROM numbers(1e5); + +DROP TABLE IF EXISTS test_table_complex; +CREATE TABLE test_table_complex +( + key_1 UInt64, + key_2 UInt64, + value UInt16 +) ENGINE=Memory() AS SELECT number, number, number FROM numbers(1e5); + +DROP DICTIONARY IF EXISTS test_sparse_dictionary_load_factor; +CREATE DICTIONARY test_sparse_dictionary_load_factor +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT(SPARSE_HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SHOW CREATE test_sparse_dictionary_load_factor; +SYSTEM RELOAD DICTIONARY test_sparse_dictionary_load_factor; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'test_sparse_dictionary_load_factor'; +SELECT count() FROM test_table WHERE dictGet('test_sparse_dictionary_load_factor', 'value', key) != value; +DROP DICTIONARY test_sparse_dictionary_load_factor; + +DROP DICTIONARY IF EXISTS test_dictionary_load_factor; +CREATE DICTIONARY test_dictionary_load_factor +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT(HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SHOW CREATE test_dictionary_load_factor; +SYSTEM RELOAD DICTIONARY test_dictionary_load_factor; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'test_dictionary_load_factor'; +SELECT count() FROM test_table WHERE dictGet('test_dictionary_load_factor', 'value', key) != value; +DROP DICTIONARY test_dictionary_load_factor; + +DROP DICTIONARY IF EXISTS test_dictionary_load_factor_nullable; +CREATE DICTIONARY test_dictionary_load_factor_nullable +( + key UInt64, + value Nullable(UInt16) +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table_nullable)) +LAYOUT(HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SHOW CREATE test_dictionary_load_factor_nullable; +SYSTEM RELOAD DICTIONARY test_dictionary_load_factor_nullable; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'test_dictionary_load_factor_nullable'; +SELECT count() FROM test_table_nullable WHERE dictGet('test_dictionary_load_factor_nullable', 'value', key) != value; +DROP DICTIONARY test_dictionary_load_factor_nullable; + +DROP DICTIONARY IF EXISTS test_complex_dictionary_load_factor; +CREATE DICTIONARY test_complex_dictionary_load_factor +( + key_1 UInt64, + key_2 UInt64, + value UInt16 +) PRIMARY KEY key_1, key_2 +SOURCE(CLICKHOUSE(TABLE test_table_complex)) +LAYOUT(COMPLEX_KEY_HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SYSTEM RELOAD DICTIONARY test_complex_dictionary_load_factor; +SHOW CREATE test_complex_dictionary_load_factor; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() and name = 'test_complex_dictionary_load_factor'; +SELECT count() FROM test_table_complex WHERE dictGet('test_complex_dictionary_load_factor', 'value', (key_1, key_2)) != value; +DROP DICTIONARY test_complex_dictionary_load_factor; + +DROP DICTIONARY IF EXISTS test_dictionary_load_factor_string; +CREATE DICTIONARY test_dictionary_load_factor_string +( + key String, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table_string)) +LAYOUT(HASHED(MAX_LOAD_FACTOR 1)) +LIFETIME(0); +-- should because of MAX_LOAD_FACTOR is 1 (maximum allowed value is 0.99) +SYSTEM RELOAD DICTIONARY test_dictionary_load_factor_string; -- { serverError BAD_ARGUMENTS } +DROP DICTIONARY test_dictionary_load_factor_string; + +DROP TABLE test_table; +DROP TABLE test_table_nullable; +DROP TABLE test_table_string; +DROP TABLE test_table_complex; diff --git a/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference new file mode 100644 index 00000000000..abe891cbb9b --- /dev/null +++ b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference @@ -0,0 +1,4 @@ +test_dictionary_hashed 1000000 0.4768 33558760 +test_dictionary_hashed_load_factor 1000000 0.9537 16781544 +test_dictionary_sparse_hashed 1000000 0.4768 20975848 +test_dictionary_sparse_hashed_load_factor 1000000 0.9537 10490088 diff --git a/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 new file mode 100644 index 00000000000..870acd54514 --- /dev/null +++ b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 @@ -0,0 +1,39 @@ +{# vi: ft=sql #} + +{% for layout in ["hashed", "sparse_hashed"] %} +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}; +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}_load_factor; +DROP TABLE IF EXISTS test_table; + +CREATE TABLE test_table +( + key UInt64, + value UInt16 +) ENGINE=Memory() AS SELECT number, number FROM numbers(1e6); + +CREATE DICTIONARY test_dictionary_{{layout}} +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT({{layout}}()) +LIFETIME(0); + +CREATE DICTIONARY test_dictionary_{{layout}}_load_factor +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT({{layout}}(MAX_LOAD_FACTOR 0.98)) +LIFETIME(0); + +SYSTEM RELOAD DICTIONARY test_dictionary_{{layout}}; +SYSTEM RELOAD DICTIONARY test_dictionary_{{layout}}_load_factor; +SELECT name, element_count, round(load_factor, 4), bytes_allocated FROM system.dictionaries WHERE database = currentDatabase() ORDER BY name; + +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}; +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}_load_factor; +DROP TABLE test_table; +{% endfor %} From 7c8d8eeb5677a22e324c1c2dd36c4036d21bce0e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 2 May 2023 20:55:47 +0200 Subject: [PATCH 058/129] Use Cell::setMapped() over separate helper insertSetMapped() Signed-off-by: Azat Khuzhin --- src/Common/HashTable/FixedHashTable.h | 2 +- src/Common/HashTable/HashTable.h | 15 ++------------- src/Common/HashTable/TwoLevelHashTable.h | 2 +- 3 files changed, 4 insertions(+), 15 deletions(-) diff --git a/src/Common/HashTable/FixedHashTable.h b/src/Common/HashTable/FixedHashTable.h index 7df90fd98b9..49675aaafbc 100644 --- a/src/Common/HashTable/FixedHashTable.h +++ b/src/Common/HashTable/FixedHashTable.h @@ -358,7 +358,7 @@ public: std::pair res; emplace(Cell::getKey(x), res.first, res.second); if (res.second) - insertSetMapped(res.first->getMapped(), x); + res.first->setMapped(x); return res; } diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 0e8510d1c8d..a48cb8d49f2 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -117,7 +117,7 @@ inline bool bitEquals(T && a, T && b) * 3) Hash tables that store the key and do not have a "mapped" value, e.g. the normal HashTable. * GetKey returns the key, and GetMapped returns a zero void pointer. This simplifies generic * code that works with mapped values: it can overload on the return type of GetMapped(), and - * doesn't need other parameters. One example is insertSetMapped() function. + * doesn't need other parameters. One example is Cell::setMapped() function. * * 4) Hash tables that store both the key and the "mapped" value, e.g. HashMap. Both GetKey and * GetMapped are supported. @@ -216,17 +216,6 @@ struct HashTableCell }; -/** - * A helper function for HashTable::insert() to set the "mapped" value. - * Overloaded on the mapped type, does nothing if it's VoidMapped. - */ -template -void insertSetMapped(VoidMapped /* dest */, const ValueType & /* src */) {} - -template -void insertSetMapped(MappedType & dest, const ValueType & src) { dest = src.second; } - - /** Determines the size of the hash table, and when and how much it should be resized. * Has very small state (one UInt8) and useful for Set-s allocated in automatic memory (see uniqExact as an example). */ @@ -1046,7 +1035,7 @@ public: } if (res.second) - insertSetMapped(res.first->getMapped(), x); + res.first->setMapped(x); return res; } diff --git a/src/Common/HashTable/TwoLevelHashTable.h b/src/Common/HashTable/TwoLevelHashTable.h index bd4c4c366f2..a60655e4072 100644 --- a/src/Common/HashTable/TwoLevelHashTable.h +++ b/src/Common/HashTable/TwoLevelHashTable.h @@ -224,7 +224,7 @@ public: emplace(Cell::getKey(x), res.first, res.second, hash_value); if (res.second) - insertSetMapped(res.first->getMapped(), x); + res.first->setMapped(x); return res; } From 65dd87d0da27feea18a3a5ea8b94c24213209357 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 2 May 2023 17:01:20 +0200 Subject: [PATCH 059/129] Fix "reference binding to misaligned address" in PackedHashMap Use separate helpers that accept/return values, instead of reference, anyway PackedHashMap is developed for small structure. v0: fix for keys v2: fix for values v3: fix bitEquals v4: fix for iterating over HashMap Signed-off-by: Azat Khuzhin --- base/base/StringRef.h | 12 ++++++ src/Common/HashTable/PackedHashMap.h | 63 +++++++++++++++++++++++++++- 2 files changed, 74 insertions(+), 1 deletion(-) diff --git a/base/base/StringRef.h b/base/base/StringRef.h index a3e32ff5058..f428b7c747f 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -3,6 +3,7 @@ #include #include // for std::logic_error #include +#include #include #include #include @@ -326,5 +327,16 @@ namespace ZeroTraits inline void set(StringRef & x) { x.size = 0; } } +namespace PackedZeroTraits +{ + template class PackedPairNoInit> + inline bool check(const PackedPairNoInit p) + { return 0 == p.key.size; } + + template class PackedPairNoInit> + inline void set(PackedPairNoInit & p) + { p.key.size = 0; } +} + std::ostream & operator<<(std::ostream & os, const StringRef & str); diff --git a/src/Common/HashTable/PackedHashMap.h b/src/Common/HashTable/PackedHashMap.h index 6b52a04aa49..6432c299b9d 100644 --- a/src/Common/HashTable/PackedHashMap.h +++ b/src/Common/HashTable/PackedHashMap.h @@ -30,6 +30,67 @@ struct PackedPairNoInit } } __attribute__((packed)); +/// The difference with ZeroTraits is that PackedZeroTraits accepts PackedPairNoInit instead of Key. +namespace PackedZeroTraits +{ + template class PackedPairNoInit> + bool check(const PackedPairNoInit p) { return p.first == First{}; } + + template class PackedPairNoInit> + void set(PackedPairNoInit & p) { p.first = First{}; } +} + +/// setZero() should be overwritten to pass the pair instead of key, to avoid +/// "reference binding to misaligned address" errors from UBsan. +template +struct PackedHashMapCell : public HashMapCell> +{ + using Base = HashMapCell>; + using State = typename Base::State; + using value_type = typename Base::value_type; + using key_type = typename Base::key_type; + using Mapped = typename Base::Mapped; + + using Base::Base; + + void setZero() { PackedZeroTraits::set(this->value); } + + Key getKey() const { return this->value.first; } + static Key getKey(const value_type & value_) { return value_.first; } + + Mapped & getMapped() { return this->value.second; } + Mapped getMapped() const { return this->value.second; } + value_type getValue() const { return this->value; } + + bool keyEquals(const Key key_) const { return bitEqualsByValue(this->value.first, key_); } + bool keyEquals(const Key key_, size_t /*hash_*/) const { return bitEqualsByValue(this->value.first, key_); } + bool keyEquals(const Key key_, size_t /*hash_*/, const State & /*state*/) const { return bitEqualsByValue(this->value.first, key_); } + + bool isZero(const State & state) const { return isZero(this->value.first, state); } + static bool isZero(const Key key, const State & /*state*/) { return ZeroTraits::check(key); } + + static inline bool bitEqualsByValue(key_type a, key_type b) { return a == b; } + + template + auto get() const + { + if constexpr (I == 0) return this->value.first; + else if constexpr (I == 1) return this->value.second; + } +}; + +namespace std +{ + template + struct tuple_size> : std::integral_constant { }; + + template + struct tuple_element<0, PackedHashMapCell> { using type = Key; }; + + template + struct tuple_element<1, PackedHashMapCell> { using type = TMapped; }; +} + /// Packed HashMap - HashMap with structure without padding /// /// Sometimes padding in structure can be crucial, consider the following @@ -43,4 +104,4 @@ template < typename Hash = DefaultHash, typename Grower = HashTableGrower<>, typename Allocator = HashTableAllocator> -using PackedHashMap = HashMapTable>, Hash, Grower, Allocator>; +using PackedHashMap = HashMapTable, Hash, Grower, Allocator>; From fc19e79f50686b7ed94165ac7827b088611a5bf7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 18:14:40 +0200 Subject: [PATCH 060/129] Change coding style of declaring packed attribute in PackedHashMap Signed-off-by: Azat Khuzhin --- src/Common/HashTable/PackedHashMap.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/HashTable/PackedHashMap.h b/src/Common/HashTable/PackedHashMap.h index 6432c299b9d..0d25addb58e 100644 --- a/src/Common/HashTable/PackedHashMap.h +++ b/src/Common/HashTable/PackedHashMap.h @@ -9,7 +9,7 @@ /// NOTE: makePairNoInit() is omitted for PackedPairNoInit since it is not /// required for PackedHashMap (see mergeBlockWithPipe() for details) template -struct PackedPairNoInit +struct __attribute__((packed)) PackedPairNoInit { First first; Second second; @@ -28,7 +28,7 @@ struct PackedPairNoInit , second(std::forward(second_)) { } -} __attribute__((packed)); +}; /// The difference with ZeroTraits is that PackedZeroTraits accepts PackedPairNoInit instead of Key. namespace PackedZeroTraits From 422cbe08fe83ca83f8ce24c48a811a1a0202abdb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 12:46:26 +0200 Subject: [PATCH 061/129] Do not use PackedHashMap for non-POD for the purposes of layout In clang-16 the behaviour for POD types had been changed in [1], this does not allows us to use PackedHashMap for some types. [1]: https://github.com/llvm/llvm-project/commit/277123376ce08c98b07c154bf83e4092a5d4d3c6 Note, that I tried to come up with a more generic solution then enumeratic types, but failed. Though now I think that this is good, since this shows which types are not allowed for PackedHashMap Another option is to use -fclang-abi-compat=13.0 but I doubt it is a good idea. Signed-off-by: Azat Khuzhin --- .../HashedDictionaryCollectionType.h | 36 ++++++++++++++++++- src/Interpreters/examples/hash_map.cpp | 30 ---------------- 2 files changed, 35 insertions(+), 31 deletions(-) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 1ab0ff8b0e3..23a08601fff 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -11,6 +12,33 @@ namespace DB { +/// Return true if the type is POD [1] for the purpose of layout (this is not +/// the same as STL traits has). +/// +/// [1]: https://stackoverflow.com/questions/4178175/what-are-aggregates-and-pods-and-how-why-are-they-special/4178176#4178176 +/// +/// The behaviour had been change in clang-16, see this for more details: +/// - https://github.com/llvm/llvm-project/commit/a8b0c6fa28acced71db33e80bd0b51d00422035b +/// - https://github.com/llvm/llvm-project/commit/277123376ce08c98b07c154bf83e4092a5d4d3c6 +/// - https://github.com/llvm/llvm-project/issues/62422 +/// - https://github.com/llvm/llvm-project/issues/62353 +/// - https://github.com/llvm/llvm-project/issues/62358 +template +constexpr bool isPodLayout() +{ + if constexpr (std::is_same_v) + return false; + if constexpr (std::is_same_v) + return false; + if constexpr (std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v) + return false; + if constexpr (std::is_same_v) + return false; + if constexpr (std::is_same_v || std::is_same_v) + return false; + return true; +} + /// HashMap with packed structure is better than google::sparse_hash_map if the /// pair is small, for the sizeof(std::pair) == 16, RSS for hash /// table with 1e9 elements will be: @@ -37,7 +65,13 @@ namespace DB template constexpr bool useSparseHashForHashedDictionary() { - return sizeof(PackedPairNoInit) > 16; + if constexpr (!isPodLayout()) + return true; + if constexpr (!isPodLayout()) + return true; + /// NOTE: One should not use PackedPairNoInit here since this will + /// create instantion of this type, and it could be illformed. + return sizeof(V) > 8; } /// Grower with custom fill limit/load factor (instead of default 50%). diff --git a/src/Interpreters/examples/hash_map.cpp b/src/Interpreters/examples/hash_map.cpp index c0016bf8cd8..f9fc36454f6 100644 --- a/src/Interpreters/examples/hash_map.cpp +++ b/src/Interpreters/examples/hash_map.cpp @@ -279,35 +279,5 @@ int main(int argc, char ** argv) << std::endl; } - if (argc < 3 || std::stol(argv[2]) == 7) - { - Stopwatch watch; - - PackedHashMap map; - PackedHashMap::LookupResult it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - { - new (&it->getMapped()) Value; - std::swap(it->getMapped(), value); - INIT - } - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "PackedHashMap. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - return 0; } From 7eba6def94e23d8049197d431daa15bfee4f85e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:35:02 +0200 Subject: [PATCH 062/129] Add a comment for HashTableGrowerWithPrecalculation about load factor Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index a48cb8d49f2..2ff789caad2 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -270,6 +270,7 @@ struct HashTableGrower /** Determines the size of the hash table, and when and how much it should be resized. * This structure is aligned to cache line boundary and also occupies it all. * Precalculates some values to speed up lookups and insertion into the HashTable (and thus has bigger memory footprint than HashTableGrower). + * This grower assume 0.5 load factor */ template class alignas(64) HashTableGrowerWithPrecalculation From 1ab130132cba6af42521ad35f3aaf97110e768be Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:37:59 +0200 Subject: [PATCH 063/129] Add more comments into HashedDictionaryCollectionType.h Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionaryCollectionType.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 23a08601fff..cdf29a82b66 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -160,6 +160,17 @@ public: }; static_assert(sizeof(HashTableGrowerWithMaxLoadFactor) == 64); +/// Above goes various specialisations for the hash table that will be used for +/// HASHED/SPARSE_HASHED dictionary, it could use one of the following depends +/// on the layout of the dictionary and types of key/value (for more info see +/// comments in this file): +/// - HashMap +/// - HashSet +/// - HashMapWithSavedHash +/// - HashSetWithSavedHash +/// - PackedHashMap +/// - google::sparse_hash_map + /// /// Map (dictionary with attributes) /// From 6f351851adf5c6bcd269e473339e0f99dd760f0a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:41:47 +0200 Subject: [PATCH 064/129] Rename grower to HashTableGrowerWithPrecalculationAndMaxLoadFactor Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 2 +- .../HashedDictionaryCollectionType.h | 24 +++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index fa9a846195d..3903317cff2 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -630,7 +630,7 @@ void HashedDictionary::createAttributes() const auto size = dict_struct.attributes.size(); attributes.reserve(size); - HashTableGrowerWithMaxLoadFactor grower(configuration.max_load_factor); + HashTableGrowerWithPrecalculationAndMaxLoadFactor grower(configuration.max_load_factor); for (const auto & dictionary_attribute : dict_struct.attributes) { diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index cdf29a82b66..a631813bdbc 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -85,7 +85,7 @@ constexpr bool useSparseHashForHashedDictionary() /// google::sparse_hash_map. /// /// Based on HashTableGrowerWithPrecalculation -class alignas(64) HashTableGrowerWithMaxLoadFactor +class alignas(64) HashTableGrowerWithPrecalculationAndMaxLoadFactor { static constexpr size_t initial_size_degree = 8; UInt8 size_degree = initial_size_degree; @@ -103,8 +103,8 @@ public: /// If collision resolution chains are contiguous, we can implement erase operation by moving the elements. static constexpr auto performs_linear_probing_with_single_step = true; - HashTableGrowerWithMaxLoadFactor() = default; - explicit HashTableGrowerWithMaxLoadFactor(float max_load_factor_) + HashTableGrowerWithPrecalculationAndMaxLoadFactor() = default; + explicit HashTableGrowerWithPrecalculationAndMaxLoadFactor(float max_load_factor_) : max_load_factor(max_load_factor_) { increaseSizeDegree(0); @@ -158,7 +158,7 @@ public: increaseSizeDegree(0); } }; -static_assert(sizeof(HashTableGrowerWithMaxLoadFactor) == 64); +static_assert(sizeof(HashTableGrowerWithPrecalculationAndMaxLoadFactor) == 64); /// Above goes various specialisations for the hash table that will be used for /// HASHED/SPARSE_HASHED dictionary, it could use one of the following depends @@ -185,8 +185,8 @@ struct HashedDictionaryMapType, HashTableGrowerWithMaxLoadFactor>, - HashMapWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; + HashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + HashMapWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; /// Implementations for SPARSE_HASHED layout. @@ -220,8 +220,8 @@ struct HashedDictionarySparseMapType, HashTableGrowerWithMaxLoadFactor>, - PackedHashMap, HashTableGrowerWithMaxLoadFactor>>; + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; template struct HashedDictionaryMapType @@ -244,8 +244,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithMaxLoadFactor>, - HashSetWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + HashSetWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; /// Implementation for SPARSE_HASHED. @@ -258,8 +258,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithMaxLoadFactor>, - HashSet, HashTableGrowerWithMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; } From 42eac6bfbc6ef80866a685b98c3ae89cb9e9e450 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:43:11 +0200 Subject: [PATCH 065/129] Wrap implementation helpers into HashedDictionaryImpl namespace Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 2 ++ src/Dictionaries/HashedDictionary.h | 4 ++-- src/Dictionaries/HashedDictionaryCollectionTraits.h | 5 +++++ src/Dictionaries/HashedDictionaryCollectionType.h | 5 +++++ 4 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 3903317cff2..1ddcfae99b4 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -32,6 +32,8 @@ namespace CurrentMetrics namespace DB { +using namespace HashedDictionaryImpl; + namespace ErrorCodes { extern const int BAD_ARGUMENTS; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index bff22706435..30eecb9ab09 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -134,7 +134,7 @@ public: private: template - using CollectionsHolder = std::vector::Type>; + using CollectionsHolder = std::vector::Type>; using NullableSet = HashSet>; using NullableSets = std::vector; @@ -232,7 +232,7 @@ private: BlockPtr update_field_loaded_block; std::vector> string_arenas; - std::vector::Type> no_attributes_containers; + std::vector::Type> no_attributes_containers; DictionaryHierarchicalParentToChildIndexPtr hierarchical_index; }; diff --git a/src/Dictionaries/HashedDictionaryCollectionTraits.h b/src/Dictionaries/HashedDictionaryCollectionTraits.h index 20fc882a2fe..4989664890a 100644 --- a/src/Dictionaries/HashedDictionaryCollectionTraits.h +++ b/src/Dictionaries/HashedDictionaryCollectionTraits.h @@ -10,6 +10,9 @@ namespace DB { +namespace HashedDictionaryImpl +{ + /// sparse_hash_map/sparse_hash_set template concept IsGoogleSparseHashTable = std::is_same_v void clearContainer(C & c) requires (IsBuiltinHashTable // NOLINTEND(*) } + +} diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index a631813bdbc..d1772b99c03 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -12,6 +12,9 @@ namespace DB { +namespace HashedDictionaryImpl +{ + /// Return true if the type is POD [1] for the purpose of layout (this is not /// the same as STL traits has). /// @@ -263,3 +266,5 @@ struct HashedDictionarySetType }; } + +} From 634f168a74b8451236407e0e861078bbd72723d7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:54:20 +0200 Subject: [PATCH 066/129] Introduce max_size_degree for HashTableGrower{,WithPrecalculation} Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 2ff789caad2..d4577d7814e 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -230,6 +230,8 @@ struct HashTableGrower /// If collision resolution chains are contiguous, we can implement erase operation by moving the elements. static constexpr auto performs_linear_probing_with_single_step = true; + static constexpr size_t max_size_degree = 23; + /// The size of the hash table in the cells. size_t bufSize() const { return 1ULL << size_degree; } @@ -248,7 +250,7 @@ struct HashTableGrower /// Increase the size of the hash table. void increaseSize() { - size_degree += size_degree >= 23 ? 1 : 2; + size_degree += size_degree >= max_size_degree ? 1 : 2; } /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. @@ -280,6 +282,7 @@ class alignas(64) HashTableGrowerWithPrecalculation UInt8 size_degree = initial_size_degree; size_t precalculated_mask = (1ULL << initial_size_degree) - 1; size_t precalculated_max_fill = 1ULL << (initial_size_degree - 1); + static constexpr size_t max_size_degree = 23; public: UInt8 sizeDegree() const { return size_degree; } @@ -309,7 +312,7 @@ public: bool overflow(size_t elems) const { return elems > precalculated_max_fill; } /// Increase the size of the hash table. - void increaseSize() { increaseSizeDegree(size_degree >= 23 ? 1 : 2); } + void increaseSize() { increaseSizeDegree(size_degree >= max_size_degree ? 1 : 2); } /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. void set(size_t num_elems) From 01bf041ccad7921b39013798a4b9261b1c37919e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:57:43 +0200 Subject: [PATCH 067/129] Rewrite HashTableGrower{,WithPrecalculation}::set w/o ternary operators Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index d4577d7814e..ca3e88c93a2 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -256,11 +256,12 @@ struct HashTableGrower /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. void set(size_t num_elems) { - size_degree = num_elems <= 1 - ? initial_size_degree - : ((initial_size_degree > static_cast(log2(num_elems - 1)) + 2) - ? initial_size_degree - : (static_cast(log2(num_elems - 1)) + 2)); + if (num_elems <= 1) + size_degree = initial_size_degree; + else if (initial_size_degree > static_cast(log2(num_elems - 1)) + 2) + size_degree = initial_size_degree; + else + size_degree = static_cast(log2(num_elems - 1)) + 2; } void setBufSize(size_t buf_size_) @@ -317,11 +318,12 @@ public: /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. void set(size_t num_elems) { - size_degree = num_elems <= 1 - ? initial_size_degree - : ((initial_size_degree > static_cast(log2(num_elems - 1)) + 2) - ? initial_size_degree - : (static_cast(log2(num_elems - 1)) + 2)); + if (num_elems <= 1) + size_degree = initial_size_degree; + else if (initial_size_degree > static_cast(log2(num_elems - 1)) + 2) + size_degree = initial_size_degree; + else + size_degree = static_cast(log2(num_elems - 1)) + 2; increaseSizeDegree(0); } From c9cde110cdaebb27ddfaa8792988ac9b5e5162f2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:59:43 +0200 Subject: [PATCH 068/129] Add initial degree as parameter for HashTableGrowerWithPrecalculationAndMaxLoadFactor Signed-off-by: Azat Khuzhin --- .../HashedDictionaryCollectionType.h | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index d1772b99c03..745c1716964 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -88,9 +88,9 @@ constexpr bool useSparseHashForHashedDictionary() /// google::sparse_hash_map. /// /// Based on HashTableGrowerWithPrecalculation +template class alignas(64) HashTableGrowerWithPrecalculationAndMaxLoadFactor { - static constexpr size_t initial_size_degree = 8; UInt8 size_degree = initial_size_degree; size_t precalculated_mask = (1ULL << initial_size_degree) - 1; size_t precalculated_max_fill = 1ULL << (initial_size_degree - 1); @@ -161,7 +161,7 @@ public: increaseSizeDegree(0); } }; -static_assert(sizeof(HashTableGrowerWithPrecalculationAndMaxLoadFactor) == 64); +static_assert(sizeof(HashTableGrowerWithPrecalculationAndMaxLoadFactor<>) == 64); /// Above goes various specialisations for the hash table that will be used for /// HASHED/SPARSE_HASHED dictionary, it could use one of the following depends @@ -188,8 +188,8 @@ struct HashedDictionaryMapType, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - HashMapWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + HashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + HashMapWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; /// Implementations for SPARSE_HASHED layout. @@ -223,8 +223,8 @@ struct HashedDictionarySparseMapType, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; template struct HashedDictionaryMapType @@ -247,8 +247,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - HashSetWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + HashSetWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; /// Implementation for SPARSE_HASHED. @@ -261,8 +261,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; } From f8e7d2cb1f186e60d4537f0069969ae440ccb217 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 20:02:13 +0200 Subject: [PATCH 069/129] Remove part of the HashTableGrowerWithPrecalculationAndMaxLoadFactor comment Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionaryCollectionType.h | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 745c1716964..0746d42bd1f 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -79,14 +79,6 @@ constexpr bool useSparseHashForHashedDictionary() /// Grower with custom fill limit/load factor (instead of default 50%). /// -/// It turns out that HashMap can outperform google::sparse_hash_map in case of -/// the structure size of not big, in terms of speed *and* memory. Even 99% of -/// max load factor was faster then google::sparse_hash_map in my simple tests -/// (1e9 UInt64 keys with UInt16 values, randomly distributed). -/// -/// And not to mention very high allocator memory fragmentation in -/// google::sparse_hash_map. -/// /// Based on HashTableGrowerWithPrecalculation template class alignas(64) HashTableGrowerWithPrecalculationAndMaxLoadFactor From 2b240d3721f66f1e3178be602837194e06d1bc44 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 20:10:06 +0200 Subject: [PATCH 070/129] Improve documentation for HASHED/SPARSE_HASHED/COMPLEX_KEY_HASHED/COMPLEX_KEY_SPARSE_HASHED Signed-off-by: Azat Khuzhin --- docs/en/sql-reference/dictionaries/index.md | 27 ++++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index dfda1ff9c04..712a17308ba 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -267,14 +267,16 @@ or LAYOUT(HASHED()) ``` -If `shards` greater then 1 (default is `1`) the dictionary will load data in parallel, useful if you have huge amount of elements in one dictionary. - Configuration example: ``` xml + 10 + 10000 + + + + ``` or ``` sql -LAYOUT(SPARSE_HASHED()) +LAYOUT(SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000] [MAX_LOAD_FACTOR 0.5])) ``` It is also possible to use `shards` for this type of dictionary, and again it is more important for `sparse_hashed` then for `hashed`, since `sparse_hashed` is slower. @@ -332,7 +339,7 @@ Configuration example: ``` xml - 1 + @@ -342,7 +349,7 @@ Configuration example: or ``` sql -LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) +LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000] [MAX_LOAD_FACTOR 0.5])) ``` ### complex_key_sparse_hashed @@ -354,7 +361,9 @@ Configuration example: ``` xml - 1 + + + ``` @@ -362,7 +371,7 @@ Configuration example: or ``` sql -LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) +LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000] [MAX_LOAD_FACTOR 0.5])) ``` ### hashed_array From e1e2a83a9e78b5e56476f38478895b8826e2b3dd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 20:28:28 +0200 Subject: [PATCH 071/129] Print type of the structure that will be used for HASHED/SPARSE_HASHED Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 1ddcfae99b4..83d051df5e9 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -660,6 +660,11 @@ void HashedDictionary::createAttributes() container.max_load_factor(configuration.max_load_factor); attributes.emplace_back(std::move(attribute)); } + + if constexpr (IsBuiltinHashTable::value_type>) + LOG_TRACE(log, "Using builtin hash table for {} attribute", dictionary_attribute.name); + else + LOG_TRACE(log, "Using sparsehash for {} attribute", dictionary_attribute.name); }; callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); From e37e8f83bb28dc752c0264cee52dcbb7c4f24352 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 16 May 2023 15:57:26 +0200 Subject: [PATCH 072/129] Fix flakiness of test_distributed_load_balancing MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit I saw the following in the logs for the failed test: 2023.05.16 07:12:12.894051 [ 262 ] {74575ac0-b296-4fdc-bc8e-3476a305e6ea} ConnectionPoolWithFailover: Connection failed at try №1, reason: Timeout exceeded while reading from socket (socket (172.16.3.2:9000), receive timeout 2000 ms) And I think that the culprit is the test_distributed_replica_max_ignored_errors for which it is normal, however not for others, and this should not affect other tests. So fix this by calling SYSTEM RELOAD CONFIG, which should reset error count. CI: https://s3.amazonaws.com/clickhouse-test-reports/49380/5abc1a1c68ee204c9024493be1d19835cf5630f7/integration_tests__release__[3_4].html Signed-off-by: Azat Khuzhin --- .../test_distributed_load_balancing/test.py | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index 1dba6a30bc4..271828f433e 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -29,23 +29,19 @@ nodes = len(cluster.instances) queries = nodes * 10 +# SYSTEM RELOAD CONFIG will reset some attributes of the nodes in cluster +# - error_count +# - last_used (round_robing) +# +# This is required to avoid interference results of one test to another +@pytest.fixture(scope="function", autouse=True) +def test_setup(): + for n in list(cluster.instances.values()): + n.query("SYSTEM RELOAD CONFIG") + + def bootstrap(): for n in list(cluster.instances.values()): - # At startup, server loads configuration files. - # - # However ConfigReloader does not know about already loaded files - # (files is empty()), hence it will always reload the configuration - # just after server starts (+ 2 seconds, reload timeout). - # - # And on configuration reload the clusters will be re-created, so some - # internal stuff will be reset: - # - error_count - # - last_used (round_robing) - # - # And if the reload will happen during round_robin test it will start - # querying from the beginning, so let's issue config reload just after - # start to avoid reload in the middle of the test execution. - n.query("SYSTEM RELOAD CONFIG") n.query("DROP TABLE IF EXISTS data") n.query("DROP TABLE IF EXISTS dist") n.query("CREATE TABLE data (key Int) Engine=Memory()") From dc353faf44c466cd89391a50076979d6f933e1d1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 16 May 2023 15:59:22 +0200 Subject: [PATCH 073/129] Simplify obtaining query shard in test_distributed_load_balancing Signed-off-by: Azat Khuzhin --- .../test_distributed_load_balancing/test.py | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index 271828f433e..e879f09ccc1 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -109,19 +109,14 @@ def get_node(query_node, table="dist", *args, **kwargs): rows = query_node.query( """ - SELECT c.host_name - FROM ( - SELECT _shard_num - FROM cluster(shards_cluster, system.query_log) - WHERE - initial_query_id = '{query_id}' AND - is_initial_query = 0 AND - type = 'QueryFinish' - ORDER BY event_date DESC, event_time DESC - LIMIT 1 - ) a - JOIN system.clusters c - ON a._shard_num = c.shard_num WHERE cluster = 'shards_cluster' + SELECT hostName() + FROM cluster(shards_cluster, system.query_log) + WHERE + initial_query_id = '{query_id}' AND + is_initial_query = 0 AND + type = 'QueryFinish' + ORDER BY event_date DESC, event_time DESC + LIMIT 1 """.format( query_id=query_id ) From f47375d16c1c89c2ffea087617f1b4dd23734f00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 May 2023 10:28:13 +0200 Subject: [PATCH 074/129] Support Tableau --- programs/server/config.xml | 8 ++++++-- src/IO/Resource/DynamicResourceManager.cpp | 4 ++-- .../queries/0_stateless/02737_sql_auto_is_null.reference | 1 + tests/queries/0_stateless/02737_sql_auto_is_null.sql | 2 ++ 4 files changed, 11 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02737_sql_auto_is_null.reference create mode 100644 tests/queries/0_stateless/02737_sql_auto_is_null.sql diff --git a/programs/server/config.xml b/programs/server/config.xml index 82dd697084c..c6ae47b3050 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -719,8 +719,12 @@ default - - + + SQL_ diff --git a/src/IO/Resource/DynamicResourceManager.cpp b/src/IO/Resource/DynamicResourceManager.cpp index 49e12984e33..df0de6575f4 100644 --- a/src/IO/Resource/DynamicResourceManager.cpp +++ b/src/IO/Resource/DynamicResourceManager.cpp @@ -98,9 +98,9 @@ DynamicResourceManager::State::Resource::~Resource() if (attached_to != nullptr) { ISchedulerNode * root = nodes.find("/")->second.ptr.get(); - attached_to->event_queue->enqueue([scheduler = attached_to, root] + attached_to->event_queue->enqueue([my_scheduler = attached_to, root] { - scheduler->removeChild(root); + my_scheduler->removeChild(root); }); } } diff --git a/tests/queries/0_stateless/02737_sql_auto_is_null.reference b/tests/queries/0_stateless/02737_sql_auto_is_null.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02737_sql_auto_is_null.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02737_sql_auto_is_null.sql b/tests/queries/0_stateless/02737_sql_auto_is_null.sql new file mode 100644 index 00000000000..22f1a9524ca --- /dev/null +++ b/tests/queries/0_stateless/02737_sql_auto_is_null.sql @@ -0,0 +1,2 @@ +SET SQL_AUTO_IS_NULL = 0; +SELECT getSetting('SQL_AUTO_IS_NULL'); From 4dbe5b83294b4a2c23e844de68de6530c653504c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 May 2023 11:13:28 +0200 Subject: [PATCH 075/129] Support them in tests --- tests/config/config.d/custom_settings_prefixes.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/custom_settings_prefixes.xml b/tests/config/config.d/custom_settings_prefixes.xml index ec66c1a930e..9baaa4e09f3 100644 --- a/tests/config/config.d/custom_settings_prefixes.xml +++ b/tests/config/config.d/custom_settings_prefixes.xml @@ -1,4 +1,4 @@ - custom_ + SQL_,custom_ From 6e468b29e894a85b954626e24fba86a2d0a8f4fb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 10:15:06 +0000 Subject: [PATCH 076/129] Check return value of ftruncate --- src/Coordination/Changelog.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index ddedae4fa0f..7275a080252 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -279,7 +279,16 @@ private: flush(); if (log_file_settings.max_size != 0) - ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); + { + int res = -1; + do + { + res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); + } while (res < 0 && errno == EINTR); + + if (res != 0) + LOG_WARNING(log, "Could not ftruncate file. Error: {}, errno: {}", errnoToString(), errno); + } if (log_file_settings.compress_logs) compressed_buffer.reset(); From f5506210d64d39b2bddecdc32a724f78b85d47af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 May 2023 12:43:55 +0200 Subject: [PATCH 077/129] Geo types are production ready --- docs/en/operations/settings/settings.md | 11 ----------- docs/en/sql-reference/data-types/index.md | 4 ++-- docs/ru/operations/settings/settings.md | 10 ---------- docs/ru/sql-reference/data-types/geo.md | 9 --------- src/Core/Settings.h | 5 +++-- src/IO/Resource/DynamicResourceManager.cpp | 4 ++-- src/Interpreters/InterpreterCreateQuery.cpp | 15 --------------- .../parseColumnsListForTableFunction.cpp | 12 ------------ .../parseColumnsListForTableFunction.h | 2 -- tests/queries/0_stateless/01291_geo_types.sql | 2 -- .../0_stateless/02513_validate_data_types.sql | 5 ----- .../queries/0_stateless/02524_fuzz_and_fuss_2.sql | 2 +- 12 files changed, 8 insertions(+), 73 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cddde2090f8..dfc433717b7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3213,17 +3213,6 @@ Possible values: Default value: `0`. -## allow_experimental_geo_types {#allow-experimental-geo-types} - -Allows working with experimental [geo data types](../../sql-reference/data-types/geo.md). - -Possible values: - -- 0 — Working with geo data types is disabled. -- 1 — Working with geo data types is enabled. - -Default value: `0`. - ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} Adds a modifier `SYNC` to all `DROP` and `DETACH` queries. diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index 2ad8ac4bb23..88663968e50 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -29,5 +29,5 @@ ClickHouse data types include: - **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type. - **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type) - **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses -- **Geo types**: for[ geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` -- **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) \ No newline at end of file +- **Geo types**: for [geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` +- **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 066e71c25a5..fa3ea582c55 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3185,16 +3185,6 @@ SELECT * FROM test2; Значение по умолчанию: `0`. -## allow_experimental_geo_types {#allow-experimental-geo-types} - -Разрешает использование экспериментальных типов данных для работы с [географическими структурами](../../sql-reference/data-types/geo.md). - -Возможные значения: -- 0 — использование типов данных для работы с географическими структурами не поддерживается. -- 1 — использование типов данных для работы с географическими структурами поддерживается. - -Значение по умолчанию: `0`. - ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} Добавляет модификатор `SYNC` ко всем запросам `DROP` и `DETACH`. diff --git a/docs/ru/sql-reference/data-types/geo.md b/docs/ru/sql-reference/data-types/geo.md index 24d981195f5..52c7dd97da0 100644 --- a/docs/ru/sql-reference/data-types/geo.md +++ b/docs/ru/sql-reference/data-types/geo.md @@ -8,13 +8,8 @@ sidebar_label: Географические структуры ClickHouse поддерживает типы данных для отображения географических объектов — точек (местоположений), территорий и т.п. -:::danger "Предупреждение" - Сейчас использование типов данных для работы с географическими структурами является экспериментальной возможностью. Чтобы использовать эти типы данных, включите настройку `allow_experimental_geo_types = 1`. -::: - **См. также** - [Хранение географических структур данных](https://ru.wikipedia.org/wiki/GeoJSON). -- Настройка [allow_experimental_geo_types](../../operations/settings/settings.md#allow-experimental-geo-types). ## Point {#point-data-type} @@ -25,7 +20,6 @@ ClickHouse поддерживает типы данных для отображ Запрос: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_point (p Point) ENGINE = Memory(); INSERT INTO geo_point VALUES((10, 10)); SELECT p, toTypeName(p) FROM geo_point; @@ -47,7 +41,6 @@ SELECT p, toTypeName(p) FROM geo_point; Запрос: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_ring (r Ring) ENGINE = Memory(); INSERT INTO geo_ring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]); SELECT r, toTypeName(r) FROM geo_ring; @@ -69,7 +62,6 @@ SELECT r, toTypeName(r) FROM geo_ring; Запись в этой таблице описывает многоугольник с одной дырой: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_polygon (pg Polygon) ENGINE = Memory(); INSERT INTO geo_polygon VALUES([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]]); SELECT pg, toTypeName(pg) FROM geo_polygon; @@ -92,7 +84,6 @@ SELECT pg, toTypeName(pg) FROM geo_polygon; Запись в этой таблице описывает элемент, состоящий из двух многоугольников — первый без дыр, а второй с одной дырой: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_multipolygon (mpg MultiPolygon) ENGINE = Memory(); INSERT INTO geo_multipolygon VALUES([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); SELECT mpg, toTypeName(mpg) FROM geo_multipolygon; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c65958b86b7..3e1400c8528 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -19,7 +19,7 @@ class IColumn; /** List of settings: type, name, default value, description, flags * - * This looks rather unconvenient. It is done that way to avoid repeating settings in different places. + * This looks rather inconvenient. It is done that way to avoid repeating settings in different places. * Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field, * but we are not going to do it, because settings is used everywhere as static struct fields. * @@ -541,7 +541,6 @@ class IColumn; M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ - M(Bool, allow_experimental_geo_types, true, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, cast_ipv4_ipv6_default_on_conversion_error, false, "CAST operator into IPv4, CAST operator into IPV6 type, toIPv4, toIPv6 functions will return default value instead of throwing exception on conversion error.", 0) \ @@ -768,6 +767,8 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_bigint_types, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_window_functions, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_lightweight_delete, true) \ + MAKE_OBSOLETE(M, Bool, allow_experimental_geo_types, true) \ + \ MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \ MAKE_OBSOLETE(M, HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT) \ MAKE_OBSOLETE(M, Bool, database_replicated_ddl_output, true) \ diff --git a/src/IO/Resource/DynamicResourceManager.cpp b/src/IO/Resource/DynamicResourceManager.cpp index 49e12984e33..df0de6575f4 100644 --- a/src/IO/Resource/DynamicResourceManager.cpp +++ b/src/IO/Resource/DynamicResourceManager.cpp @@ -98,9 +98,9 @@ DynamicResourceManager::State::Resource::~Resource() if (attached_to != nullptr) { ISchedulerNode * root = nodes.find("/")->second.ptr.get(); - attached_to->event_queue->enqueue([scheduler = attached_to, root] + attached_to->event_queue->enqueue([my_scheduler = attached_to, root] { - scheduler->removeChild(root); + my_scheduler->removeChild(root); }); } } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a34bfa5a579..3e76ad23c30 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -848,21 +848,6 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } - if (!create.attach && !settings.allow_experimental_geo_types) - { - for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) - { - const auto & type = name_and_type_pair.type->getName(); - if (type == "MultiPolygon" || type == "Polygon" || type == "Ring" || type == "Point") - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column '{}' which type is '{}' " - "because experimental geo types are not allowed. " - "Set setting allow_experimental_geo_types = 1 in order to allow it", - name_and_type_pair.name, type); - } - } - } - if (!create.attach && !settings.allow_experimental_object_type) { for (const auto & [name, type] : properties.columns.getAllPhysical()) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index e7302b6324a..05d28bc53a8 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -35,18 +35,6 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings } } - if (!settings.allow_experimental_geo_types) - { - const auto & type_name = type->getName(); - if (type_name == "MultiPolygon" || type_name == "Polygon" || type_name == "Ring" || type_name == "Point") - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Cannot create column with type '{}' because experimental geo types are not allowed. Set setting " - "allow_experimental_geo_types = 1 in order to allow it", type_name); - } - } - if (!settings.allow_experimental_object_type) { if (type->hasDynamicSubcolumns()) diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index 14119ab55da..ef1bbe5498e 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -16,14 +16,12 @@ struct DataTypeValidationSettings explicit DataTypeValidationSettings(const Settings & settings) : allow_suspicious_low_cardinality_types(settings.allow_suspicious_low_cardinality_types) - , allow_experimental_geo_types(settings.allow_experimental_geo_types) , allow_experimental_object_type(settings.allow_experimental_object_type) , allow_suspicious_fixed_string_types(settings.allow_suspicious_fixed_string_types) { } bool allow_suspicious_low_cardinality_types = true; - bool allow_experimental_geo_types = true; bool allow_experimental_object_type = true; bool allow_suspicious_fixed_string_types = true; }; diff --git a/tests/queries/0_stateless/01291_geo_types.sql b/tests/queries/0_stateless/01291_geo_types.sql index 6b686ddf520..4038c545679 100644 --- a/tests/queries/0_stateless/01291_geo_types.sql +++ b/tests/queries/0_stateless/01291_geo_types.sql @@ -1,7 +1,5 @@ DROP TABLE IF EXISTS geo; -SET allow_experimental_geo_types = 1; - CREATE TABLE geo (a Point, b Ring, c Polygon, d MultiPolygon) ENGINE=Memory(); INSERT INTO geo VALUES((0, 0), [(0, 0), (10, 0), (10, 10), (0, 10)], [[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], [[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); diff --git a/tests/queries/0_stateless/02513_validate_data_types.sql b/tests/queries/0_stateless/02513_validate_data_types.sql index 1235d00ba79..5eb91ac7879 100644 --- a/tests/queries/0_stateless/02513_validate_data_types.sql +++ b/tests/queries/0_stateless/02513_validate_data_types.sql @@ -5,10 +5,6 @@ select CAST('{"x" : 1}', 'JSON'); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONAsObject); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONEachRow, 'x JSON'); -- {serverError ILLEGAL_COLUMN} -set allow_experimental_geo_types=0; -select CAST([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], 'Polygon'); -- {serverError ILLEGAL_COLUMN} -desc file(nonexist.json, JSONEachRow, 'pg Polygon'); -- {serverError ILLEGAL_COLUMN} - set allow_suspicious_low_cardinality_types=0; select CAST(1000000, 'LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} desc file(nonexist.json, JSONEachRow, 'lc LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} @@ -16,4 +12,3 @@ desc file(nonexist.json, JSONEachRow, 'lc LowCardinality(UInt64)'); -- {serverEr set allow_suspicious_fixed_string_types=0; select CAST('', 'FixedString(1000)'); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONEachRow, 'fs FixedString(1000)'); -- {serverError ILLEGAL_COLUMN} - diff --git a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql index f0d8d5202db..d78e8d34232 100644 --- a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql +++ b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql @@ -9,6 +9,6 @@ ENGINE = Memory; INSERT INTO data_a_02187 SELECT * FROM system.one -SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', insert_distributed_sync = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '3', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_monotonous_functions_in_order_by = '1', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', allow_experimental_geo_types = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', allow_experimental_map_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; +SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', insert_distributed_sync = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '3', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_monotonous_functions_in_order_by = '1', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', allow_experimental_map_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; DROP TABLE data_a_02187; From e46476dba22af226d1ac47355e0adeaa3759d2db Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 12:44:20 +0200 Subject: [PATCH 078/129] Update src/Coordination/Changelog.cpp Co-authored-by: alesapin --- src/Coordination/Changelog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 7275a080252..e20fbf864eb 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -284,7 +284,8 @@ private: do { res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); - } while (res < 0 && errno == EINTR); + } + while (res < 0 && errno == EINTR); if (res != 0) LOG_WARNING(log, "Could not ftruncate file. Error: {}, errno: {}", errnoToString(), errno); From 9c3b17fa18ad14ece86ec665fc0dd5d8396f432c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 13:00:51 +0200 Subject: [PATCH 079/129] Remove whitespace --- src/Coordination/Changelog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index e20fbf864eb..894fd93cfa7 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -284,7 +284,7 @@ private: do { res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); - } + } while (res < 0 && errno == EINTR); if (res != 0) From 3107070e7699d14a6895dbdeb556b69986fc604b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 12:48:19 +0000 Subject: [PATCH 080/129] Avoid deadlock when starting table in attach thread --- .../ReplicatedMergeTreeRestartingThread.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 25 +++++--- .../test_replicated_table_attach/__init__.py | 0 .../configs/config.xml | 3 + .../test_replicated_table_attach/test.py | 58 +++++++++++++++++++ 5 files changed, 79 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_replicated_table_attach/__init__.py create mode 100644 tests/integration/test_replicated_table_attach/configs/config.xml create mode 100644 tests/integration/test_replicated_table_attach/test.py diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 8f5721e440d..5579fdf2033 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -29,6 +29,7 @@ public: void shutdown(bool part_of_full_shutdown); + void run(); private: StorageReplicatedMergeTree & storage; String log_name; @@ -43,8 +44,6 @@ private: UInt32 consecutive_check_failures = 0; /// How many consecutive checks have failed bool first_time = true; /// Activate replica for the first time. - void run(); - /// Restarts table if needed, returns false if it failed to restart replica. bool runImpl(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fcb7adbd69f..cdd43239bec 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4416,8 +4416,23 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) startBeingLeader(); - /// In this thread replica will be activated. - restarting_thread.start(); + if (from_attach_thread) + { + /// Try activating replica in current thread. + restarting_thread.run(); + } + else + { + /// Activate replica in a seperate thread. + restarting_thread.start(); + + /// Wait while restarting_thread finishing initialization. + /// NOTE It does not mean that replication is actually started after receiving this event. + /// It only means that an attempt to startup replication was made. + /// Table may be still in readonly mode if this attempt failed for any reason. + startup_event.wait(); + } + /// And this is just a callback session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]() { @@ -4425,12 +4440,6 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) restarting_thread.start(); }); - /// Wait while restarting_thread finishing initialization. - /// NOTE It does not mean that replication is actually started after receiving this event. - /// It only means that an attempt to startup replication was made. - /// Table may be still in readonly mode if this attempt failed for any reason. - startup_event.wait(); - startBackgroundMovesIfNeeded(); part_moves_between_shards_orchestrator.start(); diff --git a/tests/integration/test_replicated_table_attach/__init__.py b/tests/integration/test_replicated_table_attach/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_table_attach/configs/config.xml b/tests/integration/test_replicated_table_attach/configs/config.xml new file mode 100644 index 00000000000..b7b5d996072 --- /dev/null +++ b/tests/integration/test_replicated_table_attach/configs/config.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py new file mode 100644 index 00000000000..c09d67aab77 --- /dev/null +++ b/tests/integration/test_replicated_table_attach/test.py @@ -0,0 +1,58 @@ +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + with_zookeeper=True, + stay_alive=True, +) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_startup_with_small_bg_pool(started_cluster): + node.query( + "CREATE TABLE replicated_table (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table', 'r1') ORDER BY k" + ) + + node.query("INSERT INTO replicated_table VALUES(20, 30)") + + def assert_values(): + assert node.query("SELECT * FROM replicated_table") == "20\t30\n" + + assert_values() + node.restart_clickhouse(stop_start_wait_sec=10) + assert_values() + + node.query("DROP TABLE replicated_table SYNC") + +def test_startup_with_small_bg_pool_partitioned(started_cluster): + node.query( + "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" + ) + + node.query("INSERT INTO replicated_table_partitioned VALUES(20, 30)") + + def assert_values(): + assert node.query("SELECT * FROM replicated_table_partitioned") == "20\t30\n" + + assert_values() + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + node.restart_clickhouse(stop_start_wait_sec=20) + assert_values() + + node.query("DROP TABLE replicated_table_partitioned SYNC") From f947d1cc0ae147bc830c8200daf70295dc9aec38 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 May 2023 13:00:26 +0000 Subject: [PATCH 081/129] Automatic style fix --- tests/integration/test_replicated_table_attach/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index c09d67aab77..11365ea9163 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -13,6 +13,7 @@ node = cluster.add_instance( stay_alive=True, ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -39,6 +40,7 @@ def test_startup_with_small_bg_pool(started_cluster): node.query("DROP TABLE replicated_table SYNC") + def test_startup_with_small_bg_pool_partitioned(started_cluster): node.query( "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" From 55fc4adf055324cc9b359e57a8056025f744a317 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 19 May 2023 16:42:15 +0300 Subject: [PATCH 082/129] Update 02441_alter_delete_and_drop_column.sql --- tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql index b9b1b645e8e..9c4697362df 100644 --- a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql @@ -1,3 +1,4 @@ +-- Tags: no-replicated-database create table mut (n int, m int, k int) engine=ReplicatedMergeTree('/test/02441/{database}/mut', '1') order by n; set insert_keeper_fault_injection_probability=0; From acf71c5b9a976834202b856494f027421ee77422 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 15:48:31 +0200 Subject: [PATCH 083/129] Fix typo --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cdd43239bec..6373c164c40 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4423,7 +4423,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) } else { - /// Activate replica in a seperate thread. + /// Activate replica in a separate thread. restarting_thread.start(); /// Wait while restarting_thread finishing initialization. From 5237dd02451cb23d4101c0cecfe8bf133292103e Mon Sep 17 00:00:00 2001 From: mateng915 Date: Fri, 19 May 2023 22:06:43 +0800 Subject: [PATCH 084/129] New system table zookeeper connection (#45245) * Feature: Support new system table to show which zookeeper node be connected Description: ============ Currently we have no place to check which zk node be connected otherwise using lsof command. It not convenient Solution: ========= Implemented a new system table, system.zookeeper_host when CK Server has zk this table will show the zk node dir which connected by current CK server Noted: This table can support multi-zookeeper cluster scenario. * fixed review comments * added test case * update test cases * remove unused code * fixed review comments and removed unused code * updated test cases for print host, port and is_expired * modify the code comments * fixed CI Failed * fixed code style check failure * updated test cases by added Tags * update test reference * update test cases * added system.zookeeper_connection doc * Update docs/en/operations/system-tables/zookeeper_connection.md * Update docs/en/operations/system-tables/zookeeper_connection.md * Update docs/en/operations/system-tables/zookeeper_connection.md --------- Co-authored-by: Alexander Tokmakov --- .../system-tables/zookeeper_connection.md | 29 +++++++++++ src/Common/ZooKeeper/IKeeper.h | 2 + src/Common/ZooKeeper/TestKeeper.h | 3 ++ src/Common/ZooKeeper/ZooKeeper.cpp | 20 +++++++ src/Common/ZooKeeper/ZooKeeper.h | 8 +++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++ src/Common/ZooKeeper/ZooKeeperImpl.h | 3 ++ src/Interpreters/Context.cpp | 11 ++++ src/Interpreters/Context.h | 8 ++- .../StorageSystemZooKeeperConnection.cpp | 52 +++++++++++++++++++ .../System/StorageSystemZooKeeperConnection.h | 28 ++++++++++ src/Storages/System/attachSystemTables.cpp | 4 ++ ...2735_system_zookeeper_connection.reference | 1 + .../02735_system_zookeeper_connection.sql | 13 +++++ 14 files changed, 185 insertions(+), 1 deletion(-) create mode 100644 docs/en/operations/system-tables/zookeeper_connection.md create mode 100644 src/Storages/System/StorageSystemZooKeeperConnection.cpp create mode 100644 src/Storages/System/StorageSystemZooKeeperConnection.h create mode 100644 tests/queries/0_stateless/02735_system_zookeeper_connection.reference create mode 100644 tests/queries/0_stateless/02735_system_zookeeper_connection.sql diff --git a/docs/en/operations/system-tables/zookeeper_connection.md b/docs/en/operations/system-tables/zookeeper_connection.md new file mode 100644 index 00000000000..9438cda1808 --- /dev/null +++ b/docs/en/operations/system-tables/zookeeper_connection.md @@ -0,0 +1,29 @@ +--- +slug: /en/operations/system-tables/zookeeper_connection +--- +#zookeeper_connection + +This table does not exist if ZooKeeper is not configured. The 'system.zookeeper_connection' table shows current connections to ZooKeeper (including auxiliary ZooKeepers). Each row shows information about one connection. + +Columns: + +- `name` ([String](../../sql-reference/data-types/string.md)) — ZooKeeper cluster's name. +- `host` ([String](../../sql-reference/data-types/string.md)) — The hostname/IP of the ZooKeeper node that ClickHouse connected to. +- `port` ([String](../../sql-reference/data-types/string.md)) — The port of the ZooKeeper node that ClickHouse connected to. +- `index` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The index of the ZooKeeper node that ClickHouse connected to. The index is from ZooKeeper config. +- `connected_time` ([String](../../sql-reference/data-types/string.md)) — When the connection was established +- `is_expired` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Is the current connection expired. +- `keeper_api_version` ([String](../../sql-reference/data-types/string.md)) — Keeper API version. +- `client_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Session id of the connection. + +Example: + +``` sql +SELECT * FROM system.zookeeper_connection; +``` + +``` text +┌─name──────────────┬─host─────────┬─port─┬─index─┬──────connected_time─┬─is_expired─┬─keeper_api_version─┬──────────client_id─┐ +│ default_zookeeper │ 127.0.0.1 │ 2181 │ 0 │ 2023-05-19 14:30:16 │ 0 │ 0 │ 216349144108826660 │ +└───────────────────┴──────────────┴──────┴───────┴─────────────────────┴────────────┴────────────────────┴────────────────────┘ +``` diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index b09f096d761..86f9a388644 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -466,6 +466,8 @@ public: /// Useful to check owner of ephemeral node. virtual int64_t getSessionID() const = 0; + virtual String getConnectedAddress() const = 0; + /// If the method will throw an exception, callbacks won't be called. /// /// After the method is executed successfully, you must wait for callbacks diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 27405d8d571..11e56daf6b4 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -39,6 +39,7 @@ public: bool isExpired() const override { return expired; } int64_t getSessionID() const override { return 0; } + String getConnectedAddress() const override { return connected_zk_address; } void create( @@ -126,6 +127,8 @@ private: zkutil::ZooKeeperArgs args; + String connected_zk_address; + std::mutex push_request_mutex; std::atomic expired{false}; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 09047b5b232..c423e4fd498 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -111,6 +111,26 @@ void ZooKeeper::init(ZooKeeperArgs args_) LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ",")); else LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); + + String address = impl->getConnectedAddress(); + + size_t colon_pos = address.find(':'); + connected_zk_host = address.substr(0, colon_pos); + connected_zk_port = address.substr(colon_pos + 1); + + connected_zk_index = 0; + + if (args.hosts.size() > 1) + { + for (size_t i = 0; i < args.hosts.size(); i++) + { + if (args.hosts[i] == address) + { + connected_zk_index = i; + break; + } + } + } } else if (args.implementation == "testkeeper") { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index ca6a44c4cbc..9b85938c726 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -523,6 +523,10 @@ public: void setServerCompletelyStarted(); + String getConnectedZooKeeperHost() const { return connected_zk_host; } + String getConnectedZooKeeperPort() const { return connected_zk_port; } + size_t getConnectedZooKeeperIndex() const { return connected_zk_index; } + private: void init(ZooKeeperArgs args_); @@ -586,6 +590,10 @@ private: ZooKeeperArgs args; + String connected_zk_host; + String connected_zk_port; + size_t connected_zk_index; + std::mutex mutex; Poco::Logger * log = nullptr; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 6c79fc4f178..34be8aa1332 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -433,6 +433,8 @@ void ZooKeeper::connect( } connected = true; + connected_zk_address = node.address.toString(); + break; } catch (...) @@ -448,6 +450,8 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; + connected_zk_address = ""; + message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; for (const auto & node : nodes) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index c0c57d3f719..6715607ca88 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -125,6 +125,8 @@ public: /// Useful to check owner of ephemeral node. int64_t getSessionID() const override { return session_id; } + String getConnectedAddress() const override { return connected_zk_address; } + void executeGenericRequest( const ZooKeeperRequestPtr & request, ResponseCallback callback); @@ -201,6 +203,7 @@ public: private: ACLs default_acls; + String connected_zk_address; zkutil::ZooKeeperArgs args; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0ef9ea53ee8..d9f450191bc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2778,6 +2778,17 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const return zookeeper->second; } + +std::map Context::getAuxiliaryZooKeepers() const +{ + std::lock_guard lock(shared->auxiliary_zookeepers_mutex); + + if (!shared->auxiliary_zookeepers.empty()) + return shared->auxiliary_zookeepers; + else + return std::map(); +} + #if USE_ROCKSDB MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87843a458e8..15f2ff625ef 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -32,7 +32,11 @@ namespace Poco::Net { class IPAddress; } -namespace zkutil { class ZooKeeper; } +namespace zkutil +{ + class ZooKeeper; + using ZooKeeperPtr = std::shared_ptr; +} struct OvercommitTracker; @@ -827,6 +831,8 @@ public: std::shared_ptr getZooKeeper() const; /// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry. std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; + /// return Auxiliary Zookeeper map + std::map getAuxiliaryZooKeepers() const; /// Try to connect to Keeper using get(Auxiliary)ZooKeeper. Useful for /// internal Keeper start (check connection to some other node). Return true diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp new file mode 100644 index 00000000000..f249097654e --- /dev/null +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + {"host", std::make_shared()}, + {"port", std::make_shared()}, + {"index", std::make_shared()}, + {"connected_time", std::make_shared()}, + {"is_expired", std::make_shared()}, + {"keeper_api_version", std::make_shared()}, + {"client_id", std::make_shared()} + }; +} + +void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, ContextPtr context, + const SelectQueryInfo &) const +{ + res_columns[0]->insert("default_zookeeper"); + res_columns[1]->insert(context->getZooKeeper()->getConnectedZooKeeperHost()); + res_columns[2]->insert(context->getZooKeeper()->getConnectedZooKeeperPort()); + res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); + res_columns[4]->insert(context->getZooKeeperSessionUptime()); + res_columns[5]->insert(context->getZooKeeper()->expired()); + res_columns[6]->insert(context->getZooKeeper()->getApiVersion()); + res_columns[7]->insert(context->getZooKeeper()->getClientID()); + + for (const auto & elem : context->getAuxiliaryZooKeepers()) + { + res_columns[0]->insert(elem.first); + res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); + res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); + res_columns[2]->insert(elem.second->getConnectedZooKeeperPort()); + res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); + res_columns[4]->insert(elem.second->getSessionUptime()); + res_columns[5]->insert(elem.second->expired()); + res_columns[6]->insert(elem.second->getApiVersion()); + res_columns[7]->insert(elem.second->getClientID()); + } + +} + +} diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.h b/src/Storages/System/StorageSystemZooKeeperConnection.h new file mode 100644 index 00000000000..dd4c293c112 --- /dev/null +++ b/src/Storages/System/StorageSystemZooKeeperConnection.h @@ -0,0 +1,28 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/** Implements `zookeeper_connection` system table, which allows you to get information about the connected zookeeper info. + */ +class StorageSystemZooKeeperConnection final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemZooKeeperConnection"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} + diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index d6982ba30d5..424c74662ec 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -80,6 +80,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -186,7 +187,10 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "named_collections"); if (has_zookeeper) + { attach(context, system_database, "zookeeper"); + attach(context, system_database, "zookeeper_connection"); + } if (context->getConfigRef().getInt("allow_experimental_transactions", 0)) attach(context, system_database, "transactions"); diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference new file mode 100644 index 00000000000..c9cc8adede8 --- /dev/null +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -0,0 +1 @@ +[ :1]:9181 0 diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql new file mode 100644 index 00000000000..10f12177b2e --- /dev/null +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -0,0 +1,13 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS test_zk_connection_table; + +CREATE TABLE test_zk_connection_table ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02731_zk_connection/{shard}', '{replica}') +ORDER BY tuple(); + +select host, port, is_expired from system.zookeeper_connection where name='default_zookeeper'; + +DROP TABLE IF EXISTS test_zk_connection_table; From b29edc473786a6c3cd5dcc37a53fa36c32283782 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 16:38:14 +0200 Subject: [PATCH 085/129] Add method --- src/IO/S3/Client.cpp | 16 ++++++++++++---- src/IO/S3/Client.h | 9 +++++++-- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 93ef30a927e..668b1a3959d 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -133,11 +133,12 @@ ProviderType deduceProviderType(const std::string & url) Client::Client( size_t max_redirects_, ServerSideEncryptionKMSConfig sse_kms_config_, - const std::shared_ptr & credentials_provider, + const std::shared_ptr & credentials_provider_, const Aws::Client::ClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, bool use_virtual_addressing) - : Aws::S3::S3Client(credentials_provider, client_configuration, std::move(sign_payloads), use_virtual_addressing) + : Aws::S3::S3Client(credentials_provider_, client_configuration, std::move(sign_payloads), use_virtual_addressing) + , credentials_provider(credentials_provider_) , max_redirects(max_redirects_) , sse_kms_config(std::move(sse_kms_config_)) , log(&Poco::Logger::get("S3Client")) @@ -177,6 +178,7 @@ Client::Client( Client::Client(const Client & other) : Aws::S3::S3Client(other) , initial_endpoint(other.initial_endpoint) + , credentials_provider(other.credentials_provider) , explicit_region(other.explicit_region) , detect_region(other.detect_region) , provider_type(other.provider_type) @@ -188,6 +190,11 @@ Client::Client(const Client & other) ClientCacheRegistry::instance().registerClient(cache); } +Aws::Auth::AWSCredentials Client::getCredentials() const +{ + return credentials_provider->GetAWSCredentials(); +} + bool Client::checkIfWrongRegionDefined(const std::string & bucket, const Aws::S3::S3Error & error, std::string & region) const { if (detect_region) @@ -711,7 +718,8 @@ std::unique_ptr ClientFactory::create( // NOLINT const String & server_side_encryption_customer_key_base64, ServerSideEncryptionKMSConfig sse_kms_config, HTTPHeaderEntries headers, - CredentialsConfiguration credentials_configuration) + CredentialsConfiguration credentials_configuration, + const String & session_token) { PocoHTTPClientConfiguration client_configuration = cfg_; client_configuration.updateSchemeAndRegion(); @@ -735,7 +743,7 @@ std::unique_ptr ClientFactory::create( // NOLINT // These will be added after request signing client_configuration.extra_headers = std::move(headers); - Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key); + Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key, session_token); auto credentials_provider = std::make_shared( client_configuration, std::move(credentials), diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 36edb443681..e1b99c893a6 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -138,6 +138,9 @@ public: /// Returns the initial endpoint. const String & getInitialEndpoint() const { return initial_endpoint; } + const String & getRegion() const { return explicit_region; } + + Aws::Auth::AWSCredentials getCredentials() const; /// Decorator for RetryStrategy needed for this client to work correctly. /// We want to manually handle permanent moves (status code 301) because: @@ -207,7 +210,7 @@ private: Client(size_t max_redirects_, ServerSideEncryptionKMSConfig sse_kms_config_, - const std::shared_ptr& credentials_provider, + const std::shared_ptr & credentials_provider_, const Aws::Client::ClientConfiguration& client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, bool use_virtual_addressing); @@ -247,6 +250,7 @@ private: void insertRegionOverride(const std::string & bucket, const std::string & region) const; String initial_endpoint; + std::shared_ptr credentials_provider; std::string explicit_region; mutable bool detect_region = true; @@ -282,7 +286,8 @@ public: const String & server_side_encryption_customer_key_base64, ServerSideEncryptionKMSConfig sse_kms_config, HTTPHeaderEntries headers, - CredentialsConfiguration credentials_configuration); + CredentialsConfiguration credentials_configuration, + const String & session_token = ""); PocoHTTPClientConfiguration createClientConfiguration( const String & force_region, From 10a382bd12de3b31c54655587e28cd06a1123360 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 19 May 2023 10:49:43 -0400 Subject: [PATCH 086/129] add ALTER and DROP --- docs/en/operations/named-collections.md | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 1866315529e..8ad403e448e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -57,8 +57,7 @@ In the above example the `passowrd_sha256_hex` value is the hexadecimal represen ### XML example -```xml -$ cat /etc/clickhouse-server/config.d/named_collections.xml +```xml title='/etc/clickhouse-server/config.d/named_collections.xml' @@ -70,6 +69,27 @@ $ cat /etc/clickhouse-server/config.d/named_collections.xml ``` +## Modifying named collections + +Named collections that are created with DDL queries can be altered or dropped with DDL. Named collections created with XML files can be managed by editing or deleting the corresponding XML. + +### Alter a DDL named collection + +Change or add the keys `key1` and `key3` of the collection `collection2`: +```sql +ALTER NAMED COLLECTION collection2 SET key1=4, key3='value3' +``` + +Remove the key `key2` from `collection2`: +```sql +ALTER NAMED COLLECTION collection2 DELETE key2 +``` + +### Drop the DDL named collection `collection2`: +```sql +DROP NAMED COLLECTION collection2 +``` + ## Named collections for accessing S3 The description of parameters see [s3 Table Function](../sql-reference/table-functions/s3.md). @@ -389,3 +409,4 @@ SELECT dictGet('dict', 'b', 1); │ a │ └─────────────────────────┘ ``` + From ff63a73cf535fd9b18b20e6986748f8f5fc93a9f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 19 May 2023 11:03:53 -0400 Subject: [PATCH 087/129] review --- docs/en/operations/named-collections.md | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 8ad403e448e..fc2d7c3540c 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -71,7 +71,7 @@ In the above example the `passowrd_sha256_hex` value is the hexadecimal represen ## Modifying named collections -Named collections that are created with DDL queries can be altered or dropped with DDL. Named collections created with XML files can be managed by editing or deleting the corresponding XML. +Named collections that are created with DDL queries can be altered or dropped with DDL. Named collections created with XML files can be managed by editing or deleting the corresponding XML. ### Alter a DDL named collection @@ -85,6 +85,11 @@ Remove the key `key2` from `collection2`: ALTER NAMED COLLECTION collection2 DELETE key2 ``` +Change or add the key `key1` and delete the key `key3` of the collection `collection2`: +```sql +ALTER NAMED COLLECTION collection2 SET key1=4, DELETE key3 +``` + ### Drop the DDL named collection `collection2`: ```sql DROP NAMED COLLECTION collection2 From 4af818746461a59f73c3e36de291eefdbabf8e95 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 15:06:02 +0000 Subject: [PATCH 088/129] Activate restarting thread in both cases --- .../MergeTree/ReplicatedMergeTreeRestartingThread.h | 8 +++++++- src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- .../test_replicated_table_attach/configs/config.xml | 3 +++ tests/integration/test_replicated_table_attach/test.py | 3 +++ 4 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 5579fdf2033..9e99baab4c3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -23,7 +23,13 @@ class ReplicatedMergeTreeRestartingThread public: explicit ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_); - void start() { task->activateAndSchedule(); } + void start(bool schedule = true) + { + if (schedule) + task->activateAndSchedule(); + else + task->activate(); + } void wakeup() { task->schedule(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6373c164c40..eccdded9a6f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4416,6 +4416,9 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) startBeingLeader(); + /// Activate replica in a separate thread if we are not calling from attach thread + restarting_thread.start(/*schedule=*/!from_attach_thread); + if (from_attach_thread) { /// Try activating replica in current thread. @@ -4423,9 +4426,6 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) } else { - /// Activate replica in a separate thread. - restarting_thread.start(); - /// Wait while restarting_thread finishing initialization. /// NOTE It does not mean that replication is actually started after receiving this event. /// It only means that an attempt to startup replication was made. diff --git a/tests/integration/test_replicated_table_attach/configs/config.xml b/tests/integration/test_replicated_table_attach/configs/config.xml index b7b5d996072..fea3eab4126 100644 --- a/tests/integration/test_replicated_table_attach/configs/config.xml +++ b/tests/integration/test_replicated_table_attach/configs/config.xml @@ -1,3 +1,6 @@ 1 + + 5 + diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index 11365ea9163..2d209ddaf79 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -57,4 +57,7 @@ def test_startup_with_small_bg_pool_partitioned(started_cluster): node.restart_clickhouse(stop_start_wait_sec=20) assert_values() + # check that we activate it in the end + node.query_with_retry("INSERT INTO replicated_table_partitioned VALUES(20, 30)") + node.query("DROP TABLE replicated_table_partitioned SYNC") From 90872c2671f468cef899834d8fb15af6e5896960 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 19 May 2023 17:22:37 +0200 Subject: [PATCH 089/129] Update query_log.md Fix links to data types --- docs/en/operations/system-tables/query_log.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 42247e6fba2..1bcecfeb161 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -97,8 +97,8 @@ Columns: - `forwarded_for` ([String](../../sql-reference/data-types/string.md)) — HTTP header `X-Forwarded-For` passed in the HTTP query. - `quota_key` ([String](../../sql-reference/data-types/string.md)) — The `quota key` specified in the [quotas](../../operations/quotas.md) setting (see `keyed`). - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse revision. -- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — ProfileEvents that measure different metrics. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events) -- `Settings` ([Map(String, String)](../../sql-reference/data-types/array.md)) — Settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. +- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/map.md)) — ProfileEvents that measure different metrics. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events) +- `Settings` ([Map(String, String)](../../sql-reference/data-types/map.md)) — Settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. - `log_comment` ([String](../../sql-reference/data-types/string.md)) — Log comment. It can be set to arbitrary string no longer than [max_query_size](../../operations/settings/settings.md#settings-max_query_size). An empty string if it is not defined. - `thread_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Thread ids that are participating in query execution. - `used_aggregate_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `aggregate functions`, which were used during query execution. From 3e42ee7f2baf73707fdcabcd4f4afeeafbbbc14e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 17:22:57 +0200 Subject: [PATCH 090/129] Get rid of finalize callback in object storages --- src/Backups/BackupIO_S3.cpp | 1 + src/Coordination/KeeperSnapshotManagerS3.cpp | 1 + src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- src/Disks/IO/WriteBufferFromAzureBlobStorage.h | 7 +++++-- ...oteFS.cpp => WriteBufferWithFinalizeCallback.cpp} | 10 +++++----- ...mRemoteFS.h => WriteBufferWithFinalizeCallback.h} | 12 ++++++------ .../AzureBlobStorage/AzureObjectStorage.cpp | 5 +---- .../AzureBlobStorage/AzureObjectStorage.h | 2 -- .../ObjectStorages/Cached/CachedObjectStorage.cpp | 3 +-- .../ObjectStorages/Cached/CachedObjectStorage.h | 1 - .../ObjectStorages/DiskObjectStorageTransaction.cpp | 9 ++++++--- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 6 +----- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h | 1 - src/Disks/ObjectStorages/IObjectStorage.h | 3 --- .../ObjectStorages/Local/LocalObjectStorage.cpp | 6 +----- src/Disks/ObjectStorages/Local/LocalObjectStorage.h | 1 - src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 10 +++------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 - src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 2 -- src/Disks/ObjectStorages/Web/WebObjectStorage.h | 1 - src/IO/WriteBufferFromS3.cpp | 4 +++- src/IO/WriteBufferFromS3.h | 8 +++++--- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 3 ++- src/Storages/HDFS/WriteBufferFromHDFS.h | 9 +++++---- src/Storages/StorageS3.cpp | 1 + 25 files changed, 48 insertions(+), 61 deletions(-) rename src/Disks/IO/{WriteIndirectBufferFromRemoteFS.cpp => WriteBufferWithFinalizeCallback.cpp} (64%) rename src/Disks/IO/{WriteIndirectBufferFromRemoteFS.h => WriteBufferWithFinalizeCallback.h} (59%) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 40ecde71173..f1fd276e34b 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -255,6 +255,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, + DBMS_DEFAULT_BUFFER_SIZE, request_settings, std::nullopt, threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3"), diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index ffb3a0bc18b..1afe0b352c5 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -149,6 +149,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa s3_client->client, s3_client->uri.bucket, key, + DBMS_DEFAULT_BUFFER_SIZE, request_settings_1 }; }; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 44185f74f60..b5d296bd865 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -26,7 +26,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( size_t max_single_part_upload_size_, size_t buf_size_, const WriteSettings & write_settings_) - : BufferWithOwnMemory(buf_size_, nullptr, 0) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(&Poco::Logger::get("WriteBufferFromAzureBlobStorage")) , max_single_part_upload_size(max_single_part_upload_size_) , blob_path(blob_path_) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 27492809384..f1be81922e1 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -6,7 +6,7 @@ #include -#include +#include #include #include #include @@ -21,7 +21,7 @@ class Logger; namespace DB { -class WriteBufferFromAzureBlobStorage : public BufferWithOwnMemory +class WriteBufferFromAzureBlobStorage : public WriteBufferFromFileBase { public: using AzureClientPtr = std::shared_ptr; @@ -37,6 +37,9 @@ public: void nextImpl() override; + std::string getFileName() const override { return blob_path; } + void sync() override { next(); } + private: void finalizeImpl() override; void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp similarity index 64% rename from src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp rename to src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index a909e8f109c..49e230b9dc3 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -1,11 +1,11 @@ -#include "WriteIndirectBufferFromRemoteFS.h" +#include "WriteBufferWithFinalizeCallback.h" namespace DB { -WriteIndirectBufferFromRemoteFS::WriteIndirectBufferFromRemoteFS( +WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( std::unique_ptr impl_, - CreateMetadataCallback && create_callback_, + FinalizeCallback && create_callback_, const String & remote_path_) : WriteBufferFromFileDecorator(std::move(impl_)) , create_metadata_callback(std::move(create_callback_)) @@ -14,7 +14,7 @@ WriteIndirectBufferFromRemoteFS::WriteIndirectBufferFromRemoteFS( } -WriteIndirectBufferFromRemoteFS::~WriteIndirectBufferFromRemoteFS() +WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() { try { @@ -26,7 +26,7 @@ WriteIndirectBufferFromRemoteFS::~WriteIndirectBufferFromRemoteFS() } } -void WriteIndirectBufferFromRemoteFS::finalizeImpl() +void WriteBufferWithFinalizeCallback::finalizeImpl() { WriteBufferFromFileDecorator::finalizeImpl(); if (create_metadata_callback) diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h b/src/Disks/IO/WriteBufferWithFinalizeCallback.h similarity index 59% rename from src/Disks/IO/WriteIndirectBufferFromRemoteFS.h rename to src/Disks/IO/WriteBufferWithFinalizeCallback.h index f06ebe89589..73c1b8d25d4 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.h @@ -8,25 +8,25 @@ namespace DB { -using CreateMetadataCallback = std::function; +using FinalizeCallback = std::function; /// Stores data in S3/HDFS and adds the object path and object size to metadata file on local FS. -class WriteIndirectBufferFromRemoteFS final : public WriteBufferFromFileDecorator +class WriteBufferWithFinalizeCallback final : public WriteBufferFromFileDecorator { public: - WriteIndirectBufferFromRemoteFS( + WriteBufferWithFinalizeCallback( std::unique_ptr impl_, - CreateMetadataCallback && create_callback_, + FinalizeCallback && create_callback_, const String & remote_path_); - ~WriteIndirectBufferFromRemoteFS() override; + ~WriteBufferWithFinalizeCallback() override; String getFileName() const override { return remote_path; } private: void finalizeImpl() override; - CreateMetadataCallback create_metadata_callback; + FinalizeCallback create_metadata_callback; String remote_path; }; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 9db5d13a7f8..9c45c5f806e 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -129,7 +129,6 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO const StoredObject & object, WriteMode mode, std::optional, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & write_settings) { @@ -138,14 +137,12 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO LOG_TEST(log, "Writing file: {}", object.remote_path); - auto buffer = std::make_unique( + return std::make_unique( client.get(), object.remote_path, settings.get()->max_single_part_upload_size, buf_size, patchSettings(write_settings)); - - return std::make_unique(std::move(buffer), std::move(finalize_callback), object.remote_path); } void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 648016fb732..276c992e4d7 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -83,7 +82,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index a26969c6175..acf9430e85c 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -97,13 +97,12 @@ std::unique_ptr CachedObjectStorage::writeObject( /// N const StoredObject & object, WriteMode mode, // Cached doesn't support append, only rewrite std::optional attributes, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & write_settings) { /// Add cache relating settings to WriteSettings. auto modified_write_settings = IObjectStorage::patchSettings(write_settings); - auto implementation_buffer = object_storage->writeObject(object, mode, attributes, std::move(finalize_callback), buf_size, modified_write_settings); + auto implementation_buffer = object_storage->writeObject(object, mode, attributes, buf_size, modified_write_settings); bool cache_on_write = modified_write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getByName(cache_config_name).settings.cache_on_write_operations diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 276f6669bfa..f8e346e1aed 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -43,7 +43,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 2c22df64d90..257a6fdf2ea 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -658,14 +659,16 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile operations_to_execute.emplace_back(std::move(write_operation)); - /// We always use mode Rewrite because we simulate append using metadata and different files - return object_storage.writeObject( + auto impl = object_storage.writeObject( object, + /// We always use mode Rewrite because we simulate append using metadata and different files WriteMode::Rewrite, object_attributes, - std::move(create_metadata_callback), buf_size, settings); + + return std::make_unique( + std::move(impl), std::move(create_metadata_callback), object.remote_path); } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index e50e410823d..e746668dd40 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -83,7 +82,6 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL const StoredObject & object, WriteMode mode, std::optional attributes, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & write_settings) { @@ -93,11 +91,9 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL "HDFS API doesn't support custom attributes/metadata for stored objects"); /// Single O_WRONLY in libhdfs adds O_TRUNC - auto hdfs_buffer = std::make_unique( + return std::make_unique( object.remote_path, config, settings->replication, patchSettings(write_settings), buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); - - return std::make_unique(std::move(hdfs_buffer), std::move(finalize_callback), object.remote_path); } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 4064a5c5b7f..d4eac33b33e 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -81,7 +81,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index f4f1b063ade..f129bd22758 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -48,8 +48,6 @@ struct ObjectMetadata std::optional attributes; }; -using FinalizeCallback = std::function; - /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// /// Examples of object storages are S3, Azure Blob Storage, HDFS. @@ -119,7 +117,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) = 0; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 8cf0b27a517..e1e90df591b 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -124,7 +123,6 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO const StoredObject & object, WriteMode mode, std::optional /* attributes */, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & /* write_settings */) { @@ -132,9 +130,7 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO throw Exception(ErrorCodes::BAD_ARGUMENTS, "LocalObjectStorage doesn't support append to files"); LOG_TEST(log, "Write object: {}", object.remote_path); - auto impl = std::make_unique(object.remote_path, buf_size); - return std::make_unique( - std::move(impl), std::move(finalize_callback), object.remote_path); + return std::make_unique(object.remote_path, buf_size); } void LocalObjectStorage::removeObject(const StoredObject & object) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 2d31ccff398..630320ab7f9 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -41,7 +41,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79b3d3a2b8b..a82174ff1b1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -160,8 +159,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN const StoredObject & object, WriteMode mode, // S3 doesn't support append, only rewrite std::optional attributes, - FinalizeCallback && finalize_callback, - size_t buf_size [[maybe_unused]], + size_t buf_size, const WriteSettings & write_settings) { WriteSettings disk_write_settings = IObjectStorage::patchSettings(write_settings); @@ -174,17 +172,15 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); - auto s3_buffer = std::make_unique( + return std::make_unique( client.get(), bucket, object.remote_path, + buf_size, settings_ptr->request_settings, attributes, std::move(scheduler), disk_write_settings); - - return std::make_unique( - std::move(s3_buffer), std::move(finalize_callback), object.remote_path); } void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 766c1aa033d..70ed899586e 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -97,7 +97,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index c57b75f4038..4ff18383713 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -9,7 +9,6 @@ #include #include -#include #include #include #include @@ -211,7 +210,6 @@ std::unique_ptr WebObjectStorage::writeObject( /// NOLI const StoredObject & /* object */, WriteMode /* mode */, std::optional /* attributes */, - FinalizeCallback && /* finalize_callback */, size_t /* buf_size */, const WriteSettings & /* write_settings */) { diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index f55e853b4fa..e85b7224892 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -51,7 +51,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 01ab8ff7cbb..c7a3b131d09 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -79,11 +79,13 @@ WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, + size_t buf_size_, const S3Settings::RequestSettings & request_settings_, std::optional> object_metadata_, ThreadPoolCallbackRunner schedule_, const WriteSettings & write_settings_) - : bucket(bucket_) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) + , bucket(bucket_) , key(key_) , request_settings(request_settings_) , upload_settings(request_settings.getUploadSettings()) diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index ac6c430606f..f4200b0a646 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -5,7 +5,7 @@ #if USE_AWS_S3 #include -#include +#include #include #include #include @@ -24,13 +24,14 @@ namespace DB * Data is divided on chunks with size greater than 'minimum_upload_part_size'. Last chunk can be less than this threshold. * Each chunk is written as a part to S3. */ -class WriteBufferFromS3 final : public BufferWithOwnMemory +class WriteBufferFromS3 final : public WriteBufferFromFileBase { public: WriteBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, + size_t buf_size_, const S3Settings::RequestSettings & request_settings_, std::optional> object_metadata_ = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, @@ -39,8 +40,9 @@ public: ~WriteBufferFromS3() override; void nextImpl() override; void preFinalize() override; + std::string getFileName() const override { return key; } + void sync() override { next(); } -public: class IBufferAllocationPolicy { public: diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 293f5d31836..fad0447d2cf 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -107,8 +107,9 @@ WriteBufferFromHDFS::WriteBufferFromHDFS( const WriteSettings & write_settings_, size_t buf_size_, int flags_) - : BufferWithOwnMemory(buf_size_) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) , impl(std::make_unique(hdfs_name_, config_, replication_, write_settings_, flags_)) + , filename(hdfs_name_) { } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index ec54348c174..71e6e55addc 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -5,7 +5,7 @@ #if USE_HDFS #include #include -#include +#include #include #include #include @@ -17,7 +17,7 @@ namespace DB /** Accepts HDFS path to file and opens it. * Closes file by himself (thus "owns" a file descriptor). */ -class WriteBufferFromHDFS final : public BufferWithOwnMemory +class WriteBufferFromHDFS final : public WriteBufferFromFileBase { public: @@ -29,19 +29,20 @@ public: size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags = O_WRONLY); - WriteBufferFromHDFS(WriteBufferFromHDFS &&) = default; - ~WriteBufferFromHDFS() override; void nextImpl() override; void sync() override; + std::string getFileName() const override { return filename; } + private: void finalizeImpl() override; struct WriteBufferFromHDFSImpl; std::unique_ptr impl; + const std::string filename; }; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a4d9dc9f2e3..4723693ac3a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -775,6 +775,7 @@ public: configuration_.client, bucket, key, + DBMS_DEFAULT_BUFFER_SIZE, configuration_.request_settings, std::nullopt, threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelWrite"), From 4260e86b1a1dd05ba9ca65cb6cba3c61a636bb06 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 19 May 2023 13:00:18 -0400 Subject: [PATCH 091/129] fix typo --- docs/en/operations/named-collections.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index fc2d7c3540c..7db8f2b642b 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -50,7 +50,7 @@ To manage named collections with DDL a user must have the `named_control_collect ``` :::tip -In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plaintext and sha256 hex passwords set for a user. +In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plain text and sha256 hex passwords set for a user. ::: ## Storing named collections in configuration files From 35e77f8e2a8aaecc1d3328863ab38670908fab72 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 17:53:22 +0000 Subject: [PATCH 092/129] fix comment --- src/Interpreters/Cluster.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index b696b539013..8c30dbe230f 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -487,8 +487,8 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown element in config: {}", replica_key); } - addShard(settings, std::move(replica_addresses), false, current_shard_num, - std::move(insert_paths), /* treat_local_as_remote */ weight, internal_replication); + addShard(settings, std::move(replica_addresses), /* treat_local_as_remote = */ false, current_shard_num, + std::move(insert_paths), weight, internal_replication); } else throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown element in config: {}", key); From 086888b2856b52edcbca2ff9c1d949ed1c48142a Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 17:54:29 +0000 Subject: [PATCH 093/129] fix ConnectionPoolWithFailover::getPriority --- src/Client/ConnectionPoolWithFailover.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index e0c46804c60..129bc10bc27 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -73,9 +73,9 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts Int64 ConnectionPoolWithFailover::getPriority() const { - return (*std::max_element(nested_pools.begin(), nested_pools.end(), [](const auto &a, const auto &b) + return (*std::max_element(nested_pools.begin(), nested_pools.end(), [](const auto & a, const auto & b) { - return a->getPriority() - b->getPriority(); + return a->getPriority() < b->getPriority(); }))->getPriority(); } From d69c35fcdd4b183f94acffbf8fc6e73988d39ac1 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 17:57:00 +0000 Subject: [PATCH 094/129] fix PoolWithFailover `error_count` integer overflow --- src/Client/ConnectionPool.h | 1 - src/Common/PoolWithFailoverBase.h | 8 ++++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index 9b1f7dc5b9d..aacd0a063c7 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -135,7 +135,6 @@ private: Protocol::Compression compression; /// Whether to compress data when interacting with the server. Protocol::Secure secure; /// Whether to encrypt data when interacting with the server. Int64 priority; /// priority from - }; /** diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 0e8fbb4e6d1..928d8b74a3f 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -101,7 +101,7 @@ public: struct ShuffledPool { NestedPool * pool{}; - const PoolState * state{}; + const PoolState * state{}; // WARNING: valid only during initial ordering, dangling size_t index = 0; size_t error_count = 0; size_t slowdown_count = 0; @@ -115,7 +115,6 @@ public: /// this functor. The pools with lower result value will be tried first. using GetPriorityFunc = std::function; - /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. @@ -175,10 +174,11 @@ PoolWithFailoverBase::getShuffledPools( } /// Sort the pools into order in which they will be tried (based on respective PoolStates). + /// Note that `error_count` and `slowdown_count` are used for ordering, but set to zero in the resulting ShuffledPool std::vector shuffled_pools; shuffled_pools.reserve(nested_pools.size()); for (size_t i = 0; i < nested_pools.size(); ++i) - shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0}); + shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, /* error_count = */ 0, /* slowdown_count = */ 0}); ::sort( shuffled_pools.begin(), shuffled_pools.end(), [](const ShuffledPool & lhs, const ShuffledPool & rhs) @@ -371,7 +371,7 @@ PoolWithFailoverBase::updatePoolStates(size_t max_ignored_errors) /// distributed_replica_max_ignored_errors for (auto & state : result) - state.error_count = std::max(0, state.error_count - max_ignored_errors); + state.error_count = state.error_count > max_ignored_errors ? state.error_count - max_ignored_errors : 0; return result; } From 052d8aca71e0639769620dae96ae060c1b5e6e70 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 18:13:29 +0000 Subject: [PATCH 095/129] limit `max_tries` value by `max_error_cap` to avoid unlimited number of retries --- src/Common/PoolWithFailoverBase.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 928d8b74a3f..646e10d6443 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -227,6 +227,10 @@ PoolWithFailoverBase::getMany( { std::vector shuffled_pools = getShuffledPools(max_ignored_errors, get_priority); + /// Limit `max_tries` value by `max_error_cap` to avoid unlimited number of retries + if (max_tries > max_error_cap) + max_tries = max_error_cap; + /// We will try to get a connection from each pool until a connection is produced or max_tries is reached. std::vector try_results(shuffled_pools.size()); size_t entries_count = 0; From 50919e0cc611f09fa7444eeb14845a2b451305d6 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 19 May 2023 14:24:50 -0400 Subject: [PATCH 096/129] Update src/Core/Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1941c20baf..610c7135a75 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -156,7 +156,7 @@ class IColumn; M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ - M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table (up to `max_parallel_replicas`). It will work for any kind of MergeTree table.", 0) \ + M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ \ From 6fd5d8e8bab77a5265b6beb3f20b101960c2ab9c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 19 May 2023 02:07:30 +0000 Subject: [PATCH 097/129] Add setting output_format_parquet_compliant_nested_types to produce more compatible Parquet files --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 5 +++-- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/ParquetBlockOutputFormat.cpp | 11 +++++++++-- 5 files changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 69546011770..682a7569f3a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -913,6 +913,7 @@ class IColumn; M(Bool, output_format_parquet_fixed_string_as_fixed_byte_array, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary for FixedString columns.", 0) \ M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ M(ParquetCompression, output_format_parquet_compression_method, "lz4", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ + M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index fb2adff2e88..388cad54791 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,8 +80,9 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reade to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}}}, + {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, + {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, + {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 082ccb85970..dd6252b96f1 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -122,6 +122,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; + format_settings.parquet.output_compliant_nested_types = settings.output_format_parquet_compliant_nested_types; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index b9a55ffaa83..c88af650671 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -220,6 +220,7 @@ struct FormatSettings UInt64 max_block_size = 8192; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; + bool output_compliant_nested_types = true; } parquet; struct Pretty diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 0a78fd2725a..e39271efa7c 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -155,12 +155,19 @@ void ParquetBlockOutputFormat::write(Chunk chunk, size_t row_group_size) parquet::WriterProperties::Builder builder; builder.version(getParquetVersion(format_settings)); builder.compression(getParquetCompression(format_settings.parquet.output_compression_method)); - auto props = builder.build(); + + parquet::ArrowWriterProperties::Builder writer_props_builder; + if (format_settings.parquet.output_compliant_nested_types) + writer_props_builder.enable_compliant_nested_types(); + else + writer_props_builder.disable_compliant_nested_types(); + auto result = parquet::arrow::FileWriter::Open( *arrow_table->schema(), arrow::default_memory_pool(), sink, - props); + builder.build(), + writer_props_builder.build()); if (!result.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while opening a table: {}", result.status().ToString()); file_writer = std::move(result.ValueOrDie()); From 1ded6d64097dccf0585e814f2c3d522d59912b73 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 19 May 2023 16:53:26 -0400 Subject: [PATCH 098/129] Update grant.md --- docs/en/sql-reference/statements/grant.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index b6d6f285f3d..e6073f3523a 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -9,7 +9,7 @@ sidebar_label: GRANT - Grants [privileges](#grant-privileges) to ClickHouse user accounts or roles. - Assigns roles to user accounts or to the other roles. -To revoke privileges, use the [REVOKE](../../sql-reference/statements/revoke.md) statement. Also you can list granted privileges with the [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants-statement) statement. +To revoke privileges, use the [REVOKE](../../sql-reference/statements/revoke.md) statement. Also you can list granted privileges with the [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants) statement. ## Granting Privilege Syntax From 8924c17575567deaf3ae005b5edc31292417c037 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 May 2023 13:31:27 +0200 Subject: [PATCH 099/129] Fix build --- src/IO/S3/tests/gtest_aws_s3_client.cpp | 1 + src/IO/tests/gtest_writebuffer_s3.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 1e5c643d5e5..a9b5fa03f30 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -92,6 +92,7 @@ void doWriteRequest(std::shared_ptr client, const DB::S3:: client, uri.bucket, uri.key, + DBMS_DEFAULT_BUFFER_SIZE, request_settings ); diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index c0bd6742ea3..bc16af7f779 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -529,6 +529,7 @@ public: client, bucket, file_name, + DBMS_DEFAULT_BUFFER_SIZE, request_settings, std::nullopt, getAsyncPolicy().getScheduler()); From a9e834aea4bdea39748dd521306f534d9fe59c35 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 16:04:15 +0200 Subject: [PATCH 100/129] Fix 02735_system_zookeeper_connection for DatabaseReplicated Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02735_system_zookeeper_connection.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql index 10f12177b2e..b409913d183 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-replicated-database DROP TABLE IF EXISTS test_zk_connection_table; From 2444e93ad73d81ef45ca0f36e2a9478c0483d719 Mon Sep 17 00:00:00 2001 From: Yusuke Tanaka Date: Sun, 21 May 2023 14:42:17 +0900 Subject: [PATCH 101/129] Fix broken link to async_insert_deduplicate in doc --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e150060befe..791186dd9c1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1714,7 +1714,7 @@ Default value: `100000`. ### async_insert_max_query_number {#async-insert-max-query-number} -The maximum number of insert queries per block before being inserted. This setting takes effect only if [async_insert_deduplicate](#settings-async-insert-deduplicate) is enabled. +The maximum number of insert queries per block before being inserted. This setting takes effect only if [async_insert_deduplicate](#async-insert-deduplicate) is enabled. Possible values: @@ -1745,7 +1745,7 @@ Possible values: Default value: `0`. -### async_insert_deduplicate {#settings-async-insert-deduplicate} +### async_insert_deduplicate {#async-insert-deduplicate} Enables or disables insert deduplication of `ASYNC INSERT` (for Replicated\* tables). From c56e6a8b807f44419a923846c85a51c7a568b9fc Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 12:15:06 +0000 Subject: [PATCH 102/129] Add more profile events for distributconnections --- src/Client/ConnectionEstablisher.cpp | 5 +++++ src/Common/ProfileEvents.cpp | 2 ++ 2 files changed, 7 insertions(+) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 4be1475939a..98051a50eb3 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -4,6 +4,8 @@ namespace ProfileEvents { + extern const Event DistributedConnectionTries; + extern const Event DistributedConnectionUsable; extern const Event DistributedConnectionMissingTable; extern const Event DistributedConnectionStaleReplica; } @@ -35,6 +37,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: SCOPE_EXIT(is_finished = true); try { + ProfileEvents::increment(ProfileEvents::DistributedConnectionTries); result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); @@ -45,6 +48,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) { result.entry->forceConnected(*timeouts); + ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; result.is_up_to_date = true; return; @@ -65,6 +69,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: return; } + ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 82b1f289b9e..ba2a3b2afc1 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -131,6 +131,8 @@ M(ZooKeeperBytesSent, "Number of bytes send over network while communicating with ZooKeeper.") \ M(ZooKeeperBytesReceived, "Number of bytes received over network while communicating with ZooKeeper.") \ \ + M(DistributedConnectionTries, "Total count of distributed connection attempts.") \ + M(DistributedConnectionUsable, "Total count of successful distributed connections to a usable server (with required table, but maybe stale).") \ M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry.") \ M(DistributedConnectionMissingTable, "Number of times we rejected a replica from a distributed query, because it did not contain a table needed for the query.") \ M(DistributedConnectionStaleReplica, "Number of times we rejected a replica from a distributed query, because some table needed for a query had replication lag higher than the configured threshold.") \ From 44b1754ccfe364fa5d62d623e620e31ea1c10251 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 12:43:47 +0000 Subject: [PATCH 103/129] more profile events --- src/Common/PoolBase.h | 8 ++++++++ src/Common/ProfileEvents.cpp | 2 ++ 2 files changed, 10 insertions(+) diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index ba19c3e2150..8cabb472d8f 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -7,7 +7,13 @@ #include #include +#include +#include +namespace ProfileEvents +{ + extern const Event ConnectionPoolIsFullMicroseconds; +} namespace DB { @@ -144,6 +150,7 @@ public: return Entry(*items.back()); } + Stopwatch blocked; if (timeout < 0) { LOG_INFO(log, "No free connections in pool. Waiting indefinitely."); @@ -155,6 +162,7 @@ public: LOG_INFO(log, "No free connections in pool. Waiting {} ms.", timeout_ms.count()); available.wait_for(lock, timeout_ms); } + ProfileEvents::increment(ProfileEvents::ConnectionPoolIsFullMicroseconds, blocked.elapsedMicroseconds()); } } diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ba2a3b2afc1..dd96b724793 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -503,6 +503,8 @@ The server successfully detected this situation and will download merged part fr M(MergeTreeReadTaskRequestsSentElapsedMicroseconds, "Time spent in callbacks requested from the remote server back to the initiator server to choose the read task (for MergeTree tables). Measured on the remote server side.") \ M(MergeTreeAllRangesAnnouncementsSentElapsedMicroseconds, "Time spent in sending the announcement from the remote server to the initiator server about the set of data parts (for MergeTree tables). Measured on the remote server side.") \ \ + M(ConnectionPoolIsFullMicroseconds, "Total time spent waiting for a slot in connection pool.") \ + \ M(LogTest, "Number of log messages with level Test") \ M(LogTrace, "Number of log messages with level Trace") \ M(LogDebug, "Number of log messages with level Debug") \ From 0a3d986e42fc452e79b188ea047a8bbc3c132052 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 21 May 2023 20:53:33 +0800 Subject: [PATCH 104/129] Fix reporting projection broken part --- src/Storages/MergeTree/DataPartsExchange.cpp | 7 +-- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../__init__.py | 0 .../configs/testkeeper.xml | 6 ++ .../test.py | 60 +++++++++++++++++++ 6 files changed, 76 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_projection_report_broken_part/__init__.py create mode 100644 tests/integration/test_projection_report_broken_part/configs/testkeeper.xml create mode 100644 tests/integration/test_projection_report_broken_part/test.py diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 4f2ad823c3a..f2e35e2dcd2 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -130,12 +130,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write auto report_broken_part = [&]() { - if (part && part->isProjectionPart()) - { - auto parent_part = part->getParentPart()->shared_from_this(); - data.reportBrokenPart(parent_part); - } - else if (part) + if (part) data.reportBrokenPart(part); else LOG_TRACE(log, "Part {} was not found, do not report it as broken", part_name); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cf94e41a992..b21f44baeb5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7420,8 +7420,14 @@ Strings MergeTreeData::getDataPaths() const } -void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr & data_part) const +void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr data_part) const { + if (!data_part) + return; + + if (data_part->isProjectionPart()) + data_part = data_part->getParentPart()->shared_from_this(); + if (data_part->getDataPartStorage().isBroken()) { auto parts = getDataPartsForInternalUsage(); @@ -7433,7 +7439,7 @@ void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr & data_part) con broken_part_callback(part->name); } } - else if (data_part && data_part->getState() == MergeTreeDataPartState::Active) + else if (data_part->getState() == MergeTreeDataPartState::Active) broken_part_callback(data_part->name); else LOG_DEBUG(log, "Will not check potentially broken part {} because it's not active", data_part->getNameWithState()); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 04b008b623c..5488ce72631 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -718,7 +718,7 @@ public: /// Should be called if part data is suspected to be corrupted. /// Has the ability to check all other parts /// which reside on the same disk of the suspicious part. - void reportBrokenPart(MergeTreeData::DataPartPtr & data_part) const; + void reportBrokenPart(MergeTreeData::DataPartPtr data_part) const; /// TODO (alesap) Duplicate method required for compatibility. /// Must be removed. diff --git a/tests/integration/test_projection_report_broken_part/__init__.py b/tests/integration/test_projection_report_broken_part/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_projection_report_broken_part/configs/testkeeper.xml b/tests/integration/test_projection_report_broken_part/configs/testkeeper.xml new file mode 100644 index 00000000000..617371b13fa --- /dev/null +++ b/tests/integration/test_projection_report_broken_part/configs/testkeeper.xml @@ -0,0 +1,6 @@ + + + + testkeeper + + diff --git a/tests/integration/test_projection_report_broken_part/test.py b/tests/integration/test_projection_report_broken_part/test.py new file mode 100644 index 00000000000..768b6ccae4b --- /dev/null +++ b/tests/integration/test_projection_report_broken_part/test.py @@ -0,0 +1,60 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +import pytest +import time + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=[ + "configs/testkeeper.xml", + ], +) + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_projection_broken_part(): + node.query( + """ + create table test_projection_broken_parts_1 (a int, b int, projection ab (select a, sum(b) group by a)) + engine = ReplicatedMergeTree('/clickhouse-tables/test_projection_broken_parts', 'r1') + order by a settings index_granularity = 1; + + create table test_projection_broken_parts_2 (a int, b int, projection ab (select a, sum(b) group by a)) + engine ReplicatedMergeTree('/clickhouse-tables/test_projection_broken_parts', 'r2') + order by a settings index_granularity = 1; + + insert into test_projection_broken_parts_1 values (1, 1), (1, 2), (1, 3); + + system sync replica test_projection_broken_parts_2; + """ + ) + + # break projection part + node.exec_in_container( + [ + "bash", + "-c", + "rm /var/lib/clickhouse/data/default/test_projection_broken_parts_1/all_0_0_0/ab.proj/data.bin", + ] + ) + + expected_error = "No such file or directory" + assert expected_error in node.query_and_get_error( + "select sum(b) from test_projection_broken_parts_1 group by a" + ) + + time.sleep(2) + + assert int(node.query("select sum(b) from test_projection_broken_parts_1 group by a")) == 6 From 312f751503a12fc9612f071f89a48266dfb42c37 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 21 May 2023 13:08:55 +0000 Subject: [PATCH 105/129] Uppercase remaining SQL keywords --- docs/en/sql-reference/dictionaries/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index eb45247e74a..4e9bb2936db 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2206,7 +2206,7 @@ Regular expression tree dictionaries are a special type of dictionary which repr Regular expression tree dictionaries are defined in ClickHouse open-source using the YAMLRegExpTree source which is provided the path to a YAML file containing the regular expression tree. ```sql -create dictionary regexp_dict +CREATE DICTIONARY regexp_dict ( regexp String, name String, @@ -2322,7 +2322,7 @@ clickhouse client \ You can see how to [Insert Local Files](https://clickhouse.com/docs/en/integrations/data-ingestion/insert-local-files) for more details. After we initialize the source table, we can create a RegexpTree by table source: ``` sql -create dictionary regexp_dict +CREATE DICTIONARY regexp_dict ( regexp String, name String, From 9d9d4e3d62ea042952bf808143898529dd86822d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 21 May 2023 13:33:03 +0000 Subject: [PATCH 106/129] Some fixups --- docs/en/sql-reference/dictionaries/index.md | 37 ++++++++++----------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 4e9bb2936db..522fe132a66 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2201,7 +2201,7 @@ Result: Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of (user agent)[https://en.wikipedia.org/wiki/User_agent] strings, which can be expressed elegantly with regexp tree dictionaries. -### Use Regular Expression Tree Dictionary in ClickHouse Open-Source Environment +### Use Regular Expression Tree Dictionary in ClickHouse Open-Source Regular expression tree dictionaries are defined in ClickHouse open-source using the YAMLRegExpTree source which is provided the path to a YAML file containing the regular expression tree. @@ -2238,15 +2238,14 @@ The dictionary source `YAMLRegExpTree` represents the structure of a regexp tree version: '10' ``` -This config consists of a list of Regular Expression Tree nodes. Each node has following structure: +This config consists of a list of regular expression tree nodes. Each node has the following structure: -- **regexp** means the regular expression of this node. -- **user defined attributes** is a list of dictionary attributes defined in the dictionary structure. In this case, we have two attributes: `name` and `version`. The first node has both attributes. The second node only has `name` attribute, because the `version` is defined in the children nodes. - - The value of an attribute could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. -- **children nodes** is the children of a regexp tree node, which has their own attributes and children nodes. String matching preceeds in a depth-first fasion. If a string matches any regexp node in the top layer, the dictionary checks if the string matches the children nodes of it. If it matches, we assign the attributes of the matching nodes. If two or more nodes define the same attribute, children nodes have more priority. - - the name of **children nodes** in YAML files can be arbitrary. +- **regexp**: the regular expression of the node. +- **attributes**: a list of user-defined dictionary attributes. In this example, there are two attributes: `name` and `version`. The first node defines both attributes. The second node only defines attribute `name`. Attribute `version` is provided by the child nodes of the second node. + - The value of an attribute may contain **back references**, referring to capture groups of the matched regular expression. In the example, the value of attribute `version` in the first node consists of a back-reference `\1` to capture group `(\d+[\.\d]*)` in the regular expression. Back-reference numbers range from 1 to 9 and are written as `$1` or `\1` (for number 1). The back reference is replaced by the matched capture group during query execution. +- **child nodes**: a list of children of a regexp tree node, each of which has its own attributes and (potentially) children nodes. String matching proceeds in a depth-first fashion. If a string matches a regexp node, the dictionary checks if it also matches the nodes' child nodes. If that is the case, the attributes of the deepest matching node are assigned. Attributes of a child node overwrite equally named attributes of parent nodes. The name of child nodes in YAML files can be arbitrary, e.g. `versions` in above example. -Due to the specialty of regexp tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. +Regexp tree dictionaries only allow access using functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. Example: @@ -2262,16 +2261,16 @@ Result: └─────────────────────────────────────────────────────────────────┘ ``` -In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node, so the dictionary continues to look into the children nodes and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Android` defined in the first layer and the value of `version` is `12` defined the child node. +In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node. The dictionary then continues to look into the child nodes and find that the string also matches `3[12]/tclwebkit`. As a result, the value of attribute `name` is `Android` (defined in the first layer) and the value of `version` is `12` (defined the child node). -With a powerful YAML configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) +With a powerful YAML configure file, we can use a regexp tree dictionaries as a user agent string parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) ### Use Regular Expression Tree Dictionary in ClickHouse Cloud -We have shown how Regular Expression Tree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local YAML file, we can use this file to create Regular Expression Tree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. +Above used `YAMLRegExpTree` source works in ClickHouse Open Source but not in ClickHouse Cloud. To use regexp tree dictionaries in ClickHouse could, first create a regexp tree dictionary from a YAML file locally in ClickHouse Open Source, then dump this dictionary into a CSV file using the `dictionary` table function and the [INTO OUTFILE](../statements/select/into-outfile.md) clause. ```sql -select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') +SELECT * FROM dictionary(regexp_dict) INTO OUTFILE('regexp_dict.csv') ``` The content of csv file is: @@ -2285,15 +2284,15 @@ The content of csv file is: 6,2,"3[12]/tclwebkit","['version']","['10']" ``` -The schema of dumped file is always +The schema of dumped file is: -- `id UInt64` represents the id of the RegexpTree node. -- `parent_id UInt64` represents the id of the parent of a node. -- `regexp String` represents the regular expression string. -- `keys Array(String)` represents the names of user defined attributes. -- `values Array(String)` represents the values of user defined attributes. +- `id UInt64`: the id of the RegexpTree node. +- `parent_id UInt64`: the id of the parent of a node. +- `regexp String`: the regular expression string. +- `keys Array(String)`: the names of user-defined attributes. +- `values Array(String)`: the values of user-defined attributes. -On the cloud, we can create a table `regexp_dictionary_source_table` with the above table structure. +To create the dictionary in ClickHouse Cloud, first create a table `regexp_dictionary_source_table` with below table structure: ```sql CREATE TABLE regexp_dictionary_source_table From 491cf8b6e199757b35cef5273ea8c3cea76879b9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 21 May 2023 13:43:05 +0000 Subject: [PATCH 107/129] Fix minor mistakes --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 522fe132a66..f593cbe9052 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2261,7 +2261,7 @@ Result: └─────────────────────────────────────────────────────────────────┘ ``` -In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node. The dictionary then continues to look into the child nodes and find that the string also matches `3[12]/tclwebkit`. As a result, the value of attribute `name` is `Android` (defined in the first layer) and the value of `version` is `12` (defined the child node). +In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node. The dictionary then continues to look into the child nodes and finds that the string also matches `3[12]/tclwebkit`. As a result, the value of attribute `name` is `Android` (defined in the first layer) and the value of attribute `version` is `12` (defined the child node). With a powerful YAML configure file, we can use a regexp tree dictionaries as a user agent string parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) From 8a40a48a4902800306d6772bee920b980481d9b4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 May 2023 00:44:17 +0800 Subject: [PATCH 108/129] Fix python format --- .../integration/test_projection_report_broken_part/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_projection_report_broken_part/test.py b/tests/integration/test_projection_report_broken_part/test.py index 768b6ccae4b..f376adf4f1a 100644 --- a/tests/integration/test_projection_report_broken_part/test.py +++ b/tests/integration/test_projection_report_broken_part/test.py @@ -16,6 +16,7 @@ node = cluster.add_instance( ], ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: @@ -24,6 +25,7 @@ def start_cluster(): finally: cluster.shutdown() + def test_projection_broken_part(): node.query( """ @@ -57,4 +59,7 @@ def test_projection_broken_part(): time.sleep(2) - assert int(node.query("select sum(b) from test_projection_broken_parts_1 group by a")) == 6 + assert ( + int(node.query("select sum(b) from test_projection_broken_parts_1 group by a")) + == 6 + ) From 0586a274321f4f8bb852ef1cd6893ed587941d19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 18 May 2023 21:58:18 +0200 Subject: [PATCH 109/129] Charge only server memory for dictionaries Right now the memory will be counted for query/user for dictionary, but only if it load by user (via SYSTEM RELOAD QUERY or via dictGet()), but it could be also loaded in backgrounad (due to lifetime, or update_field, so it is like Buffer, only server memory should be charged. v2: mark test as long Signed-off-by: Azat Khuzhin Co-authored-by: Sergei Trifonov --- src/Dictionaries/HashedDictionary.cpp | 7 ++++ src/Interpreters/ExternalLoader.cpp | 4 +++ .../02760_dictionaries_memory.reference | 0 .../02760_dictionaries_memory.sql.j2 | 32 +++++++++++++++++++ 4 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/02760_dictionaries_memory.reference create mode 100644 tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 83d051df5e9..b528c0faa4c 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -68,6 +69,9 @@ public: shards_queues[shard].emplace(backlog); pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()] { + /// Do not account memory that was occupied by the dictionaries for the query/user context. + MemoryTrackerBlockerInThread memory_blocker; + if (thread_group) CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictLoad"); @@ -226,6 +230,9 @@ HashedDictionary::~HashedDictionary() pool.trySchedule([&container, thread_group = CurrentThread::getGroup()] { + /// Do not account memory that was occupied by the dictionaries for the query/user context. + MemoryTrackerBlockerInThread memory_blocker; + if (thread_group) CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictDtor"); diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 42258b3b912..5dee750889c 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -978,6 +979,9 @@ private: if (thread_group) CurrentThread::attachToGroup(thread_group); + /// Do not account memory that was occupied by the dictionaries for the query/user context. + MemoryTrackerBlockerInThread memory_blocker; + LOG_TRACE(log, "Start loading object '{}'", name); try { diff --git a/tests/queries/0_stateless/02760_dictionaries_memory.reference b/tests/queries/0_stateless/02760_dictionaries_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 b/tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 new file mode 100644 index 00000000000..ea979506e07 --- /dev/null +++ b/tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 @@ -0,0 +1,32 @@ +-- Tags: long +-- Tag long: in parallel runs could exceed 60 seconds +{# vim: ft=sql #} + +SET max_memory_usage=0; +DROP DICTIONARY IF EXISTS dict; +DROP TABLE IF EXISTS dict_data; + +CREATE TABLE dict_data (key UInt64, value UInt64) Engine=Memory(); +INSERT INTO dict_data SELECT number, number%10 FROM numbers(3_000_000); + +SET max_memory_usage='4Mi'; +{% for layout in [ + 'FLAT(INITIAL_ARRAY_SIZE 3_000_000 MAX_ARRAY_SIZE 3_000_000)', + 'HASHED()', + 'HASHED_ARRAY()', + 'SPARSE_HASHED()', + 'SPARSE_HASHED(SHARDS 2 /* shards are special, they use threads */)', +] %} +CREATE DICTIONARY dict (key UInt64, value UInt64) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE dict_data)) LIFETIME(0) LAYOUT({{layout}}); +SYSTEM RELOAD DICTIONARY dict; +-- assert that dictionary in memory takes more than 20MB, that way for each +-- shard we will have 10MB, that way we ensure that the allocations will be +-- definitely correct for the memory tracker to hit the MEMORY_LIMIT_EXCEEDED +-- error. +SELECT throwIf(bytes_allocated < 20e6, 'Memory constraints does not matched for layout {{layout}}') FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict' FORMAT Null; +DROP DICTIONARY dict; + +CREATE DICTIONARY dict (key UInt64, value UInt64) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE dict_data)) LIFETIME(0) LAYOUT({{layout}}); +SELECT dictGet('dict', 'value', 1::UInt64) FORMAT Null; +DROP DICTIONARY dict; +{% endfor %} From b6cc504717dacf968dd8bff0d9f76e824fd3a246 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 09:22:00 +0200 Subject: [PATCH 110/129] Remove Common/OpenTelemetryTraceContext.h from Context.h Signed-off-by: Azat Khuzhin --- src/Interpreters/Context.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 15f2ff625ef..1be662e0958 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include From 66cf16410daf872b241c5b9f65e2e95eac9f85a5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 19 May 2023 10:37:22 +0200 Subject: [PATCH 111/129] Preserve initial_query_id for ON CLUSTER queries v2: add proper escaping v3: set distributed_ddl_output_mode=none for test to fix replicated database build Signed-off-by: Azat Khuzhin --- src/Interpreters/DDLTask.cpp | 16 +++++++++++++ src/Interpreters/DDLTask.h | 4 +++- src/Interpreters/DDLWorker.cpp | 2 ++ src/Interpreters/executeDDLQueryOnCluster.cpp | 1 + .../02761_ddl_initial_query_id.reference | 5 ++++ .../0_stateless/02761_ddl_initial_query_id.sh | 24 +++++++++++++++++++ 6 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02761_ddl_initial_query_id.reference create mode 100755 tests/queries/0_stateless/02761_ddl_initial_query_id.sh diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 799f1b0b4f4..b24856a6146 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -104,6 +104,14 @@ String DDLLogEntry::toString() const if (version >= OPENTELEMETRY_ENABLED_VERSION) wb << "tracing: " << this->tracing_context; + /// NOTE: OPENTELEMETRY_ENABLED_VERSION has new line in TracingContext::serialize(), so no need to add one more + + if (version >= PRESERVE_INITIAL_QUERY_ID_VERSION) + { + writeString("initial_query_id: ", wb); + writeEscapedString(initial_query_id, wb); + writeChar('\n', wb); + } return wb.str(); } @@ -150,6 +158,14 @@ void DDLLogEntry::parse(const String & data) rb >> "tracing: " >> this->tracing_context; } + if (version >= PRESERVE_INITIAL_QUERY_ID_VERSION) + { + checkString("initial_query_id: ", rb); + readEscapedString(initial_query_id, rb); + checkChar('\n', rb); + } + + assertEOF(rb); if (!host_id_strings.empty()) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 2043de6701e..e92b1f9a885 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -71,10 +71,11 @@ struct DDLLogEntry static constexpr const UInt64 SETTINGS_IN_ZK_VERSION = 2; static constexpr const UInt64 NORMALIZE_CREATE_ON_INITIATOR_VERSION = 3; static constexpr const UInt64 OPENTELEMETRY_ENABLED_VERSION = 4; + static constexpr const UInt64 PRESERVE_INITIAL_QUERY_ID_VERSION = 5; /// Add new version here /// Remember to update the value below once new version is added - static constexpr const UInt64 DDL_ENTRY_FORMAT_MAX_VERSION = 4; + static constexpr const UInt64 DDL_ENTRY_FORMAT_MAX_VERSION = 5; UInt64 version = 1; String query; @@ -82,6 +83,7 @@ struct DDLLogEntry String initiator; // optional std::optional settings; OpenTelemetry::TracingContext tracing_context; + String initial_query_id; void setSettingsIfRequired(ContextPtr context); String toString() const; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 7c021bd82e6..81c78000ac3 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -476,6 +476,8 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep query_context->setSetting("implicit_transaction", Field{0}); } + query_context->getClientInfo().initial_query_id = task.entry.initial_query_id; + if (!task.is_initial_query) query_scope.emplace(query_context); executeQuery(istr, ostr, !task.is_initial_query, query_context, {}); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index e4d880c1e0c..04c1d545207 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -184,6 +184,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.initiator = ddl_worker.getCommonHostID(); entry.setSettingsIfRequired(context); entry.tracing_context = OpenTelemetry::CurrentContext(); + entry.initial_query_id = context->getClientInfo().initial_query_id; String node_path = ddl_worker.enqueueQuery(entry); return getDistributedDDLStatus(node_path, entry, context, /* hosts_to_wait */ nullptr); diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference new file mode 100644 index 00000000000..5c6f448eed5 --- /dev/null +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference @@ -0,0 +1,5 @@ +default distributed_ddl_entry_format_version +DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost +distributed_ddl_entry_format_version=PRESERVE_INITIAL_QUERY_ID_VERSION +DROP TABLE IF EXISTS default.foo +DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh new file mode 100755 index 00000000000..e9a315b812b --- /dev/null +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "default distributed_ddl_entry_format_version" +query_id="$(random_str 10)" +$CLICKHOUSE_CLIENT --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" +$CLICKHOUSE_CLIENT -q "SELECT query FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart'" + +echo "distributed_ddl_entry_format_version=PRESERVE_INITIAL_QUERY_ID_VERSION" +PRESERVE_INITIAL_QUERY_ID_VERSION=5 +query_id="$(random_str 10)" +# Check that serialization will not be broken with new lines in initial_query_id +query_id+=$'\nfoo' +$CLICKHOUSE_CLIENT --distributed_ddl_entry_format_version=$PRESERVE_INITIAL_QUERY_ID_VERSION --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" +# - normalizeQuery() is required to strip out DDL comment +# - replace() is required to avoid non deterministic behaviour of +# normalizeQuery() that replaces the identifier with "?" only if it has more +# then two numbers. +$CLICKHOUSE_CLIENT -q "SELECT normalizeQuery(replace(query, currentDatabase(), 'default')) FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart' ORDER BY event_time_microseconds" From ef06bb8f1409961d789d071703dfe1d1b4aef871 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 21 May 2023 23:07:03 +0200 Subject: [PATCH 112/129] Fix crashing in case of Replicated database without arguments Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- .../0_stateless/02762_replicated_database_no_args.reference | 0 .../queries/0_stateless/02762_replicated_database_no_args.sql | 4 ++++ 3 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02762_replicated_database_no_args.reference create mode 100644 tests/queries/0_stateless/02762_replicated_database_no_args.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3e76ad23c30..ab9e1fb04d6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -227,7 +227,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) metadata_path = metadata_path / "metadata" / database_name_escaped; } - if (create.storage->engine->name == "Replicated" && !internal && !create.attach) + if (create.storage->engine->name == "Replicated" && !internal && !create.attach && create.storage->engine->arguments) { /// Fill in default parameters if (create.storage->engine->arguments->children.size() == 1) diff --git a/tests/queries/0_stateless/02762_replicated_database_no_args.reference b/tests/queries/0_stateless/02762_replicated_database_no_args.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02762_replicated_database_no_args.sql b/tests/queries/0_stateless/02762_replicated_database_no_args.sql new file mode 100644 index 00000000000..1409a059b02 --- /dev/null +++ b/tests/queries/0_stateless/02762_replicated_database_no_args.sql @@ -0,0 +1,4 @@ +-- Tags: no-parallel + +set allow_experimental_database_replicated=1; +create database replicated_db_no_args engine=Replicated; -- { serverError BAD_ARGUMENTS } From 39ff030a6e9a646a90f0616c63b9b8064588436e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 May 2023 10:17:04 +0800 Subject: [PATCH 113/129] grace hash join supports right/full join --- src/Interpreters/GraceHashJoin.cpp | 79 +- src/Interpreters/GraceHashJoin.h | 3 +- src/Interpreters/JoinUtils.cpp | 2 +- .../Transforms/JoiningTransform.cpp | 87 +- src/Processors/Transforms/JoiningTransform.h | 6 +- .../02724_grace_hash_join.reference | 1804 +++++++++++++++++ .../0_stateless/02724_grace_hash_join.sql | 44 + 7 files changed, 1965 insertions(+), 60 deletions(-) create mode 100644 tests/queries/0_stateless/02724_grace_hash_join.reference create mode 100644 tests/queries/0_stateless/02724_grace_hash_join.sql diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 7795061072c..32801887f2b 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -302,10 +302,12 @@ void GraceHashJoin::initBuckets() current_bucket->startJoining(); } -bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) +bool GraceHashJoin::isSupported(const std::shared_ptr & table_join [[maybe_unused]]) { + bool is_asof = (table_join->strictness() == JoinStrictness::Asof); - return !is_asof && isInnerOrLeft(table_join->kind()) && table_join->oneDisjunct(); + auto kind = table_join->kind(); + return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); } GraceHashJoin::~GraceHashJoin() = default; @@ -325,7 +327,6 @@ bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) con /// One row can't be split, avoid loop if (total_rows < 2) return false; - bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes); if (has_overflow) @@ -471,23 +472,33 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block &, const Block &, UInt64) const +IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks( + const Block & left_sample_block_ [[maybe_unused]], const Block & result_sample_block_ [[maybe_unused]], UInt64 max_block_size_ [[maybe_unused]]) const { - /// We do no support returning non joined blocks here. - /// TODO: They _should_ be reported by getDelayedBlocks instead - return nullptr; + return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream { public: - explicit DelayedBlocks(size_t current_bucket_, Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, const Names & right_key_names_) + explicit DelayedBlocks( + size_t current_bucket_, + Buckets buckets_, + InMemoryJoinPtr hash_join_, + const Names & left_key_names_, + const Names & right_key_names_, + const Block & left_sample_block_, + const Block & result_sample_block_, + size_t max_block_size_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) + , left_sample_block(left_sample_block_) + , result_sample_block(result_sample_block_) + , max_block_size(max_block_size_) { } @@ -499,10 +510,27 @@ public: do { - block = left_reader.read(); - if (!block) + if (!is_left_reader_finished) { - return {}; + block = left_reader.read(); + if (!block) + { + is_left_reader_finished = true; + non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); + } + } + if (is_left_reader_finished) + { + if (non_joined_blocks) + { + block = non_joined_blocks->next(); + return block; + } + else + { + // left/inner join. + return {}; + } } Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); @@ -544,6 +572,11 @@ public: Names left_key_names; Names right_key_names; + Block left_sample_block; + Block result_sample_block; + bool is_left_reader_finished = false; + IBlocksStreamPtr non_joined_blocks; + size_t max_block_size; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() @@ -555,16 +588,6 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; - if (hash_join) - { - auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); - for (auto & block : right_blocks) - { - Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size()); - flushBlocksToBuckets(blocks, buckets, bucket_idx); - } - } - hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) @@ -589,8 +612,18 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() LOG_TRACE(log, "Loaded bucket {} with {}(/{}) rows", bucket_idx, hash_join->getTotalRowCount(), num_rows); - - return std::make_unique(current_bucket->idx, buckets, hash_join, left_key_names, right_key_names); + auto result_sample_block = left_sample_block; + ExtraBlockPtr tmp; + hash_join->joinBlock(result_sample_block, tmp); + return std::make_unique( + current_bucket->idx, + buckets, + hash_join, + left_key_names, + right_key_names, + left_sample_block, + result_sample_block, + max_block_size); } LOG_TRACE(log, "Finished loading all {} buckets", buckets.size()); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index b8d83f4cad0..eb39ee09208 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -13,7 +13,6 @@ namespace DB { - class TableJoin; class HashJoin; @@ -79,7 +78,7 @@ public: bool supportTotals() const override { return false; } IBlocksStreamPtr - getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size) const override; /// Open iterator over joined blocks. /// Must be called after all @joinBlock calls. diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index a05b58e14a1..59ee86ffa18 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -19,6 +19,7 @@ #include + namespace DB { @@ -864,7 +865,6 @@ Block NotJoinedBlocks::nextImpl() size_t rows_added = filler->fillColumns(columns_right); if (rows_added == 0) return {}; - addLeftColumns(result_block, rows_added); addRightColumns(result_block, columns_right); } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index bba8ec6fa16..4789e9c514a 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -350,6 +350,11 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!task) { + if (input.isFinished()) + { + output.finish(); + return Status::Finished; + } if (!input.hasData()) { input.setNeeded(); @@ -362,10 +367,14 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() output.pushException(data.exception); return Status::Finished; } - - if (!data.chunk.hasChunkInfo()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); - task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); + if (data.chunk.hasChunkInfo()) + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); + else + { + // Try to get one task from DelayedJoinedBlocksTransform again. + task = nullptr; + return Status::NeedData; + } } else { @@ -386,7 +395,6 @@ void DelayedJoinedBlocksWorkerTransform::work() { if (!task) return; - Block block = task->delayed_blocks->next(); if (!block) @@ -408,15 +416,19 @@ DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, J void DelayedJoinedBlocksTransform::work() { - if (finished) + if (all_buckets_finished) return; - delayed_blocks = join->getDelayedBlocks(); - finished = finished || delayed_blocks == nullptr; + if (!delayed_blocks) + { + all_buckets_finished = true; + } } IProcessor::Status DelayedJoinedBlocksTransform::prepare() { + + bool should_finished = false; for (auto & output : outputs) { if (output.isFinished()) @@ -424,41 +436,56 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() /// If at least one output is finished, then we have read all data from buckets. /// Some workers can still be busy with joining the last chunk of data in memory, /// but after that they also will finish when they will try to get next chunk. - finished = true; - continue; + should_finished = true; + break; } - if (!output.canPush()) - return Status::PortFull; } - - if (finished) + if (should_finished) { for (auto & output : outputs) { - if (output.isFinished()) - continue; - Chunk chunk; - chunk.setChunkInfo(std::make_shared()); - output.push(std::move(chunk)); - output.finish(); + if (!output.isFinished()) + { + output.finish(); + } } - return Status::Finished; } - if (delayed_blocks) + // No pending buckets. + if (all_buckets_finished && !delayed_blocks) { for (auto & output : outputs) - { - Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); - output.push(std::move(chunk)); - } - delayed_blocks = nullptr; - return Status::PortFull; + output.finish(); + return Status::Finished; } - return Status::Ready; + // Try to read next bucket. + if (!delayed_blocks) + return Status::Ready; + + // Put the ready task to the first available output port. + // Put empty tasks to other output ports. these ports will do nothing but require data again. + // Each bucket will handled in one thread, avoid lock contention in left file reader. + for (auto & output : outputs) + { + if (output.canPush()) + { + if (delayed_blocks) + { + Chunk chunk; + chunk.setChunkInfo(std::make_shared(delayed_blocks)); + output.push(std::move(chunk)); + delayed_blocks = nullptr; + } + else + { + Chunk chunk; + output.push(std::move(chunk)); + } + } + } + return Status::PortFull; } } diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e7edff40c56..f189f53c004 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -125,7 +125,6 @@ public: using DelayedBlocksTaskPtr = std::shared_ptr; - /// Reads delayed joined blocks from Join class DelayedJoinedBlocksTransform : public IProcessor { @@ -139,9 +138,8 @@ public: private: JoinPtr join; - - IBlocksStreamPtr delayed_blocks = nullptr; - bool finished = false; + IBlocksStreamPtr delayed_blocks; + bool all_buckets_finished = false; }; class DelayedJoinedBlocksWorkerTransform : public IProcessor diff --git a/tests/queries/0_stateless/02724_grace_hash_join.reference b/tests/queries/0_stateless/02724_grace_hash_join.reference new file mode 100644 index 00000000000..296ef623f04 --- /dev/null +++ b/tests/queries/0_stateless/02724_grace_hash_join.reference @@ -0,0 +1,1804 @@ +inner join 1 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +inner join 2 +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +left join 1 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +left join 2 +0 0 \N +1 1 \N +2 2 \N +3 3 \N +4 4 \N +5 5 \N +6 6 \N +7 7 \N +8 8 \N +9 9 \N +10 10 \N +11 11 \N +12 12 \N +13 13 \N +14 14 \N +15 15 \N +16 16 \N +17 17 \N +18 18 \N +19 19 \N +20 20 \N +21 21 \N +22 22 \N +23 23 \N +24 24 \N +25 25 \N +26 26 \N +27 27 \N +28 28 \N +29 29 \N +30 0 \N +31 1 \N +32 2 \N +33 3 \N +34 4 \N +35 5 \N +36 6 \N +37 7 \N +38 8 \N +39 9 \N +40 10 \N +41 11 \N +42 12 \N +43 13 \N +44 14 \N +45 15 \N +46 16 \N +47 17 \N +48 18 \N +49 19 \N +50 20 \N +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +200 20 \N +201 21 \N +202 22 \N +203 23 \N +204 24 \N +205 25 \N +206 26 \N +207 27 \N +208 28 \N +209 29 \N +210 0 \N +211 1 \N +212 2 \N +213 3 \N +214 4 \N +215 5 \N +216 6 \N +217 7 \N +218 8 \N +219 9 \N +220 10 \N +221 11 \N +222 12 \N +223 13 \N +224 14 \N +225 15 \N +226 16 \N +227 17 \N +228 18 \N +229 19 \N +230 20 \N +231 21 \N +232 22 \N +233 23 \N +234 24 \N +235 25 \N +236 26 \N +237 27 \N +238 28 \N +239 29 \N +240 0 \N +241 1 \N +242 2 \N +243 3 \N +244 4 \N +245 5 \N +246 6 \N +247 7 \N +248 8 \N +249 9 \N +250 10 \N +251 11 \N +252 12 \N +253 13 \N +254 14 \N +255 15 \N +256 16 \N +257 17 \N +258 18 \N +259 19 \N +260 20 \N +261 21 \N +262 22 \N +263 23 \N +264 24 \N +265 25 \N +266 26 \N +267 27 \N +268 28 \N +269 29 \N +270 0 \N +271 1 \N +272 2 \N +273 3 \N +274 4 \N +275 5 \N +276 6 \N +277 7 \N +278 8 \N +279 9 \N +280 10 \N +281 11 \N +282 12 \N +283 13 \N +284 14 \N +285 15 \N +286 16 \N +287 17 \N +288 18 \N +289 19 \N +290 20 \N +291 21 \N +292 22 \N +293 23 \N +294 24 \N +295 25 \N +296 26 \N +297 27 \N +298 28 \N +299 29 \N +right join 1 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +right join 2 +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +full join 1 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +full join 2 +0 0 \N +1 1 \N +2 2 \N +3 3 \N +4 4 \N +5 5 \N +6 6 \N +7 7 \N +8 8 \N +9 9 \N +10 10 \N +11 11 \N +12 12 \N +13 13 \N +14 14 \N +15 15 \N +16 16 \N +17 17 \N +18 18 \N +19 19 \N +20 20 \N +21 21 \N +22 22 \N +23 23 \N +24 24 \N +25 25 \N +26 26 \N +27 27 \N +28 28 \N +29 29 \N +30 0 \N +31 1 \N +32 2 \N +33 3 \N +34 4 \N +35 5 \N +36 6 \N +37 7 \N +38 8 \N +39 9 \N +40 10 \N +41 11 \N +42 12 \N +43 13 \N +44 14 \N +45 15 \N +46 16 \N +47 17 \N +48 18 \N +49 19 \N +50 20 \N +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +200 20 \N +201 21 \N +202 22 \N +203 23 \N +204 24 \N +205 25 \N +206 26 \N +207 27 \N +208 28 \N +209 29 \N +210 0 \N +211 1 \N +212 2 \N +213 3 \N +214 4 \N +215 5 \N +216 6 \N +217 7 \N +218 8 \N +219 9 \N +220 10 \N +221 11 \N +222 12 \N +223 13 \N +224 14 \N +225 15 \N +226 16 \N +227 17 \N +228 18 \N +229 19 \N +230 20 \N +231 21 \N +232 22 \N +233 23 \N +234 24 \N +235 25 \N +236 26 \N +237 27 \N +238 28 \N +239 29 \N +240 0 \N +241 1 \N +242 2 \N +243 3 \N +244 4 \N +245 5 \N +246 6 \N +247 7 \N +248 8 \N +249 9 \N +250 10 \N +251 11 \N +252 12 \N +253 13 \N +254 14 \N +255 15 \N +256 16 \N +257 17 \N +258 18 \N +259 19 \N +260 20 \N +261 21 \N +262 22 \N +263 23 \N +264 24 \N +265 25 \N +266 26 \N +267 27 \N +268 28 \N +269 29 \N +270 0 \N +271 1 \N +272 2 \N +273 3 \N +274 4 \N +275 5 \N +276 6 \N +277 7 \N +278 8 \N +279 9 \N +280 10 \N +281 11 \N +282 12 \N +283 13 \N +284 14 \N +285 15 \N +286 16 \N +287 17 \N +288 18 \N +289 19 \N +290 20 \N +291 21 \N +292 22 \N +293 23 \N +294 24 \N +295 25 \N +296 26 \N +297 27 \N +298 28 \N +299 29 \N diff --git a/tests/queries/0_stateless/02724_grace_hash_join.sql b/tests/queries/0_stateless/02724_grace_hash_join.sql new file mode 100644 index 00000000000..370f6058bef --- /dev/null +++ b/tests/queries/0_stateless/02724_grace_hash_join.sql @@ -0,0 +1,44 @@ +set join_algorithm = 'grace_hash'; +set max_rows_in_join = 100; + +drop table if exists test_t1; +drop table if exists test_t2; + +CREATE TABLE test_t1 +( + `x` UInt64, + `y` Nullable(UInt64) +) +ENGINE = Memory; + +CREATE TABLE test_t2 +( + `x` UInt64, + `y` Nullable(UInt64) +) +ENGINE = Memory; + +insert into test_t1 select number as x, number % 20 as y from numbers(200) where number > 50 order by y; +insert into test_t2 select number as x , number % 30 as y from numbers(300) order by y; + +select 'inner join 1'; +select l.x, l.y, r.y from test_t1 as l inner join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'inner join 2'; +select l.x, l.y, r.y from test_t2 as l inner join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; +select 'left join 1'; +select l.x, l.y, r.y from test_t1 as l left join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'left join 2'; +select l.x, l.y, r.y from test_t2 as l left join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; +select 'right join 1'; +select l.x, l.y, r.y from test_t1 as l right join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'right join 2'; +select l.x, l.y, r.y from test_t2 as l right join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; +select 'full join 1'; +select l.x, l.y, r.y from test_t1 as l full join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'full join 2'; +select l.x, l.y, r.y from test_t2 as l full join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; + + + + + From 39db0f84d95ade3ab88c0c6bb8cd24b4ac2a58a0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 May 2023 11:27:20 +0800 Subject: [PATCH 114/129] add comment --- src/Interpreters/GraceHashJoin.cpp | 11 ++++++++--- src/Interpreters/JoinUtils.cpp | 2 +- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 32801887f2b..95bb76b8373 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -302,7 +302,7 @@ void GraceHashJoin::initBuckets() current_bucket->startJoining(); } -bool GraceHashJoin::isSupported(const std::shared_ptr & table_join [[maybe_unused]]) +bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { bool is_asof = (table_join->strictness() == JoinStrictness::Asof); @@ -472,8 +472,10 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks( - const Block & left_sample_block_ [[maybe_unused]], const Block & result_sample_block_ [[maybe_unused]], UInt64 max_block_size_ [[maybe_unused]]) const +// This is only be called for bucket[0] at present. other buckets non-joined blocks are generated in +// DelayedBlocks. +IBlocksStreamPtr +GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } @@ -510,6 +512,8 @@ public: do { + // When left reader finish, return non-joined blocks. + // empty block means the end of this stream. if (!is_left_reader_finished) { block = left_reader.read(); @@ -521,6 +525,7 @@ public: } if (is_left_reader_finished) { + // full/right join, non_joined_blocks != nullptr if (non_joined_blocks) { block = non_joined_blocks->next(); diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index 59ee86ffa18..a05b58e14a1 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -19,7 +19,6 @@ #include - namespace DB { @@ -865,6 +864,7 @@ Block NotJoinedBlocks::nextImpl() size_t rows_added = filler->fillColumns(columns_right); if (rows_added == 0) return {}; + addLeftColumns(result_block, rows_added); addRightColumns(result_block, columns_right); } From 7772fed161b96cda58b837e589413b5fec01d49d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 5 May 2023 15:09:39 +0800 Subject: [PATCH 115/129] update 1. fixed the memoery overflow problem when handle all delayed buckets parallely 2. resue exists tests --- src/Interpreters/GraceHashJoin.cpp | 87 +- src/Interpreters/GraceHashJoin.h | 3 + .../Transforms/JoiningTransform.cpp | 90 +- src/Processors/Transforms/JoiningTransform.h | 6 +- .../01721_join_implicit_cast_long.sql.j2 | 1 - .../0_stateless/02273_full_sort_join.sql.j2 | 2 - .../02274_full_sort_join_nodistinct.sql.j2 | 1 - .../02275_full_sort_join_long.reference | 24 +- .../02275_full_sort_join_long.sql.j2 | 9 +- .../02724_grace_hash_join.reference | 1804 ----------------- .../0_stateless/02724_grace_hash_join.sql | 44 - 11 files changed, 128 insertions(+), 1943 deletions(-) delete mode 100644 tests/queries/0_stateless/02724_grace_hash_join.reference delete mode 100644 tests/queries/0_stateless/02724_grace_hash_join.sql diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 95bb76b8373..407663ac310 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -12,6 +12,7 @@ #include #include #include +#include "GraceHashJoin.h" #include #include @@ -472,12 +473,67 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -// This is only be called for bucket[0] at present. other buckets non-joined blocks are generated in +// Add lock for reading non-joined blocks, it's thread safe. +class GraceHashJoin::NonJoinedBlocksStream : public IBlocksStream +{ +public: + explicit NonJoinedBlocksStream( + InMemoryJoinPtr hash_join_, + const Block & left_sample_block_, + const Block & result_sample_block_, + size_t max_block_size_) + : hash_join(hash_join_) + , left_sample_block(left_sample_block_) + , result_sample_block(result_sample_block_) + , max_block_size(max_block_size_) + { + } + + Block nextImpl() override + { + // initialize non_joined_blocks lazily. + if (!has_init_non_joined_blocks) [[unlikely]] + { + std::lock_guard lock(mutex); + if (!has_init_non_joined_blocks) + { + has_init_non_joined_blocks = true; + non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); + } + } + // For right/full join, non_joined_blocks is not null. + // For inner/left join, non_joined_blocks is null, and return a empty block directly. + std::lock_guard lock(mutex); + if (non_joined_blocks) + return non_joined_blocks->next(); + return {}; + } + +private: + InMemoryJoinPtr hash_join; + Block left_sample_block; + Block result_sample_block; + size_t max_block_size; + bool has_init_non_joined_blocks = false; + IBlocksStreamPtr non_joined_blocks = nullptr; + std::mutex mutex; +}; + +// This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in // DelayedBlocks. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); + if (!has_initialized_non_joined_blocks) [[unlikely]] + { + std::lock_guard lock(hash_join_mutex); + if (!has_initialized_non_joined_blocks) + { + has_initialized_non_joined_blocks = true; + non_joined_blocks = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + } + } + return non_joined_blocks; } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -498,12 +554,14 @@ public: , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) - , left_sample_block(left_sample_block_) - , result_sample_block(result_sample_block_) - , max_block_size(max_block_size_) + , non_joined_blocks_iter(hash_join, left_sample_block_, result_sample_block_, max_block_size_) { } + // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransforms, need locks for + // - reading from left_reader. left_reader.read() has had a lock inside. + // - reading non-joined blocks from hash_join. Since iterate on non-joined blocks will has state + // changed inside. Block nextImpl() override { Block block; @@ -520,24 +578,14 @@ public: if (!block) { is_left_reader_finished = true; - non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); } } if (is_left_reader_finished) { - // full/right join, non_joined_blocks != nullptr - if (non_joined_blocks) - { - block = non_joined_blocks->next(); - return block; - } - else - { - // left/inner join. - return {}; - } + return non_joined_blocks_iter.next(); } + // block comes from left_reader, need to join with right table to get the result. Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); block = std::move(blocks[current_idx]); @@ -577,11 +625,8 @@ public: Names left_key_names; Names right_key_names; - Block left_sample_block; - Block result_sample_block; bool is_left_reader_finished = false; - IBlocksStreamPtr non_joined_blocks; - size_t max_block_size; + NonJoinedBlocksStream non_joined_blocks_iter; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index eb39ee09208..ec0e5ac05aa 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -43,6 +43,7 @@ class HashJoin; class GraceHashJoin final : public IJoin { class FileBucket; + class NonJoinedBlocksStream; class DelayedBlocks; using InMemoryJoin = HashJoin; @@ -144,6 +145,8 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; + mutable bool has_initialized_non_joined_blocks = false; + mutable std::shared_ptr non_joined_blocks = nullptr; }; } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 4789e9c514a..b638a5582c5 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -350,11 +350,6 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!task) { - if (input.isFinished()) - { - output.finish(); - return Status::Finished; - } if (!input.hasData()) { input.setNeeded(); @@ -367,14 +362,10 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() output.pushException(data.exception); return Status::Finished; } - if (data.chunk.hasChunkInfo()) - task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); - else - { - // Try to get one task from DelayedJoinedBlocksTransform again. - task = nullptr; - return Status::NeedData; - } + + if (!data.chunk.hasChunkInfo()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else { @@ -395,6 +386,7 @@ void DelayedJoinedBlocksWorkerTransform::work() { if (!task) return; + Block block = task->delayed_blocks->next(); if (!block) @@ -416,19 +408,15 @@ DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, J void DelayedJoinedBlocksTransform::work() { - if (all_buckets_finished) + if (finished) return; + delayed_blocks = join->getDelayedBlocks(); - if (!delayed_blocks) - { - all_buckets_finished = true; - } + finished = finished || delayed_blocks == nullptr; } IProcessor::Status DelayedJoinedBlocksTransform::prepare() { - - bool should_finished = false; for (auto & output : outputs) { if (output.isFinished()) @@ -436,56 +424,44 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() /// If at least one output is finished, then we have read all data from buckets. /// Some workers can still be busy with joining the last chunk of data in memory, /// but after that they also will finish when they will try to get next chunk. - should_finished = true; - break; + finished = true; + continue; } - } - if (should_finished) - { - for (auto & output : outputs) - { - if (!output.isFinished()) - { - output.finish(); - } - } - return Status::Finished; + if (!output.canPush()) + return Status::PortFull; } - // No pending buckets. - if (all_buckets_finished && !delayed_blocks) + if (finished) { + // Since have memory limit, cannot handle all buckets parallelly by different + // DelayedJoinedBlocksWorkerTransform. So send the same task to all outputs. + // Wait for all DelayedJoinedBlocksWorkerTransform be idle before getting next bucket. for (auto & output : outputs) + { + if (output.isFinished()) + continue; + Chunk chunk; + chunk.setChunkInfo(std::make_shared()); + output.push(std::move(chunk)); output.finish(); + } + return Status::Finished; } - // Try to read next bucket. - if (!delayed_blocks) - return Status::Ready; - - // Put the ready task to the first available output port. - // Put empty tasks to other output ports. these ports will do nothing but require data again. - // Each bucket will handled in one thread, avoid lock contention in left file reader. - for (auto & output : outputs) + if (delayed_blocks) { - if (output.canPush()) + for (auto & output : outputs) { - if (delayed_blocks) - { - Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); - output.push(std::move(chunk)); - delayed_blocks = nullptr; - } - else - { - Chunk chunk; - output.push(std::move(chunk)); - } + Chunk chunk; + chunk.setChunkInfo(std::make_shared(delayed_blocks)); + output.push(std::move(chunk)); } + delayed_blocks = nullptr; + return Status::PortFull; } - return Status::PortFull; + + return Status::Ready; } } diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index f189f53c004..e7edff40c56 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -125,6 +125,7 @@ public: using DelayedBlocksTaskPtr = std::shared_ptr; + /// Reads delayed joined blocks from Join class DelayedJoinedBlocksTransform : public IProcessor { @@ -138,8 +139,9 @@ public: private: JoinPtr join; - IBlocksStreamPtr delayed_blocks; - bool all_buckets_finished = false; + + IBlocksStreamPtr delayed_blocks = nullptr; + bool finished = false; }; class DelayedJoinedBlocksWorkerTransform : public IProcessor diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index f5321939f28..38f71f4c5ec 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -10,7 +10,6 @@ INSERT INTO t1 SELECT number as a, 100 + number as b FROM system.numbers LIMIT 1 INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers LIMIT 1, 10; {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 8b739330364..b371d9544da 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -26,9 +26,7 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); - {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 613da65421e..6bf634ca1f5 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -16,7 +16,6 @@ INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2 INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference index 9ec06aea3e6..73482358d12 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02275_full_sort_join_long.reference @@ -41,16 +41,34 @@ ALL INNER ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 98cc46c9cb4..0b28fd67050 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -22,11 +22,6 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; -{% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } -SELECT 'skipped'; -{% endif -%} -{% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +35,6 @@ SET join_algorithm = '{{ join_algorithm }}'; SET max_block_size = {{ block_size }}; -{% if not (kind == 'ANY' and join_algorithm == 'grace_hash') -%} SELECT '{{ kind }} INNER'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 @@ -58,9 +52,8 @@ SELECT '{{ kind }} RIGHT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 {{ kind }} RIGHT JOIN t2 ON t1.key == t2.key -; {{ is_implemented(join_algorithm) }} +; -{% endif -%} {% endfor -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02724_grace_hash_join.reference b/tests/queries/0_stateless/02724_grace_hash_join.reference deleted file mode 100644 index 296ef623f04..00000000000 --- a/tests/queries/0_stateless/02724_grace_hash_join.reference +++ /dev/null @@ -1,1804 +0,0 @@ -inner join 1 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -inner join 2 -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -left join 1 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -left join 2 -0 0 \N -1 1 \N -2 2 \N -3 3 \N -4 4 \N -5 5 \N -6 6 \N -7 7 \N -8 8 \N -9 9 \N -10 10 \N -11 11 \N -12 12 \N -13 13 \N -14 14 \N -15 15 \N -16 16 \N -17 17 \N -18 18 \N -19 19 \N -20 20 \N -21 21 \N -22 22 \N -23 23 \N -24 24 \N -25 25 \N -26 26 \N -27 27 \N -28 28 \N -29 29 \N -30 0 \N -31 1 \N -32 2 \N -33 3 \N -34 4 \N -35 5 \N -36 6 \N -37 7 \N -38 8 \N -39 9 \N -40 10 \N -41 11 \N -42 12 \N -43 13 \N -44 14 \N -45 15 \N -46 16 \N -47 17 \N -48 18 \N -49 19 \N -50 20 \N -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -200 20 \N -201 21 \N -202 22 \N -203 23 \N -204 24 \N -205 25 \N -206 26 \N -207 27 \N -208 28 \N -209 29 \N -210 0 \N -211 1 \N -212 2 \N -213 3 \N -214 4 \N -215 5 \N -216 6 \N -217 7 \N -218 8 \N -219 9 \N -220 10 \N -221 11 \N -222 12 \N -223 13 \N -224 14 \N -225 15 \N -226 16 \N -227 17 \N -228 18 \N -229 19 \N -230 20 \N -231 21 \N -232 22 \N -233 23 \N -234 24 \N -235 25 \N -236 26 \N -237 27 \N -238 28 \N -239 29 \N -240 0 \N -241 1 \N -242 2 \N -243 3 \N -244 4 \N -245 5 \N -246 6 \N -247 7 \N -248 8 \N -249 9 \N -250 10 \N -251 11 \N -252 12 \N -253 13 \N -254 14 \N -255 15 \N -256 16 \N -257 17 \N -258 18 \N -259 19 \N -260 20 \N -261 21 \N -262 22 \N -263 23 \N -264 24 \N -265 25 \N -266 26 \N -267 27 \N -268 28 \N -269 29 \N -270 0 \N -271 1 \N -272 2 \N -273 3 \N -274 4 \N -275 5 \N -276 6 \N -277 7 \N -278 8 \N -279 9 \N -280 10 \N -281 11 \N -282 12 \N -283 13 \N -284 14 \N -285 15 \N -286 16 \N -287 17 \N -288 18 \N -289 19 \N -290 20 \N -291 21 \N -292 22 \N -293 23 \N -294 24 \N -295 25 \N -296 26 \N -297 27 \N -298 28 \N -299 29 \N -right join 1 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -right join 2 -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -full join 1 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -full join 2 -0 0 \N -1 1 \N -2 2 \N -3 3 \N -4 4 \N -5 5 \N -6 6 \N -7 7 \N -8 8 \N -9 9 \N -10 10 \N -11 11 \N -12 12 \N -13 13 \N -14 14 \N -15 15 \N -16 16 \N -17 17 \N -18 18 \N -19 19 \N -20 20 \N -21 21 \N -22 22 \N -23 23 \N -24 24 \N -25 25 \N -26 26 \N -27 27 \N -28 28 \N -29 29 \N -30 0 \N -31 1 \N -32 2 \N -33 3 \N -34 4 \N -35 5 \N -36 6 \N -37 7 \N -38 8 \N -39 9 \N -40 10 \N -41 11 \N -42 12 \N -43 13 \N -44 14 \N -45 15 \N -46 16 \N -47 17 \N -48 18 \N -49 19 \N -50 20 \N -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -200 20 \N -201 21 \N -202 22 \N -203 23 \N -204 24 \N -205 25 \N -206 26 \N -207 27 \N -208 28 \N -209 29 \N -210 0 \N -211 1 \N -212 2 \N -213 3 \N -214 4 \N -215 5 \N -216 6 \N -217 7 \N -218 8 \N -219 9 \N -220 10 \N -221 11 \N -222 12 \N -223 13 \N -224 14 \N -225 15 \N -226 16 \N -227 17 \N -228 18 \N -229 19 \N -230 20 \N -231 21 \N -232 22 \N -233 23 \N -234 24 \N -235 25 \N -236 26 \N -237 27 \N -238 28 \N -239 29 \N -240 0 \N -241 1 \N -242 2 \N -243 3 \N -244 4 \N -245 5 \N -246 6 \N -247 7 \N -248 8 \N -249 9 \N -250 10 \N -251 11 \N -252 12 \N -253 13 \N -254 14 \N -255 15 \N -256 16 \N -257 17 \N -258 18 \N -259 19 \N -260 20 \N -261 21 \N -262 22 \N -263 23 \N -264 24 \N -265 25 \N -266 26 \N -267 27 \N -268 28 \N -269 29 \N -270 0 \N -271 1 \N -272 2 \N -273 3 \N -274 4 \N -275 5 \N -276 6 \N -277 7 \N -278 8 \N -279 9 \N -280 10 \N -281 11 \N -282 12 \N -283 13 \N -284 14 \N -285 15 \N -286 16 \N -287 17 \N -288 18 \N -289 19 \N -290 20 \N -291 21 \N -292 22 \N -293 23 \N -294 24 \N -295 25 \N -296 26 \N -297 27 \N -298 28 \N -299 29 \N diff --git a/tests/queries/0_stateless/02724_grace_hash_join.sql b/tests/queries/0_stateless/02724_grace_hash_join.sql deleted file mode 100644 index 370f6058bef..00000000000 --- a/tests/queries/0_stateless/02724_grace_hash_join.sql +++ /dev/null @@ -1,44 +0,0 @@ -set join_algorithm = 'grace_hash'; -set max_rows_in_join = 100; - -drop table if exists test_t1; -drop table if exists test_t2; - -CREATE TABLE test_t1 -( - `x` UInt64, - `y` Nullable(UInt64) -) -ENGINE = Memory; - -CREATE TABLE test_t2 -( - `x` UInt64, - `y` Nullable(UInt64) -) -ENGINE = Memory; - -insert into test_t1 select number as x, number % 20 as y from numbers(200) where number > 50 order by y; -insert into test_t2 select number as x , number % 30 as y from numbers(300) order by y; - -select 'inner join 1'; -select l.x, l.y, r.y from test_t1 as l inner join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'inner join 2'; -select l.x, l.y, r.y from test_t2 as l inner join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; -select 'left join 1'; -select l.x, l.y, r.y from test_t1 as l left join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'left join 2'; -select l.x, l.y, r.y from test_t2 as l left join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; -select 'right join 1'; -select l.x, l.y, r.y from test_t1 as l right join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'right join 2'; -select l.x, l.y, r.y from test_t2 as l right join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; -select 'full join 1'; -select l.x, l.y, r.y from test_t1 as l full join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'full join 2'; -select l.x, l.y, r.y from test_t2 as l full join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; - - - - - From d89beb1bf7ddfa782eb7420aa5e148de5237ffc6 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 6 May 2023 09:05:23 +0800 Subject: [PATCH 116/129] update tests --- src/Interpreters/GraceHashJoin.cpp | 15 ++----- src/Interpreters/GraceHashJoin.h | 2 - ...01721_join_implicit_cast_long.reference.j2 | 42 ------------------- .../02273_full_sort_join.reference.j2 | 16 ------- ...274_full_sort_join_nodistinct.reference.j2 | 32 -------------- 5 files changed, 3 insertions(+), 104 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 407663ac310..ee369d4b195 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -524,16 +524,7 @@ private: IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - if (!has_initialized_non_joined_blocks) [[unlikely]] - { - std::lock_guard lock(hash_join_mutex); - if (!has_initialized_non_joined_blocks) - { - has_initialized_non_joined_blocks = true; - non_joined_blocks = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); - } - } - return non_joined_blocks; + return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -580,7 +571,7 @@ public: is_left_reader_finished = true; } } - if (is_left_reader_finished) + if (is_left_reader_finished || !block) { return non_joined_blocks_iter.next(); } @@ -625,7 +616,7 @@ public: Names left_key_names; Names right_key_names; - bool is_left_reader_finished = false; + std::atomic is_left_reader_finished = false; NonJoinedBlocksStream non_joined_blocks_iter; }; diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index ec0e5ac05aa..bc5f894bc8c 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -145,8 +145,6 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; - mutable bool has_initialized_non_joined_blocks = false; - mutable std::shared_ptr non_joined_blocks = nullptr; }; } diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index e9f32087439..d40fc7ec864 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,7 +1,6 @@ {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -17,7 +16,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -30,7 +28,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -41,7 +38,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -49,7 +45,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -65,7 +60,6 @@ 8 8 0 9 9 0 10 10 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -78,7 +72,6 @@ 9 9 0 10 10 0 = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -89,7 +82,6 @@ 3 3 3 4 4 4 5 5 5 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -98,7 +90,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -114,7 +105,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -127,7 +117,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -138,7 +127,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -146,7 +134,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -162,7 +149,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -175,7 +161,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -186,7 +171,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -196,7 +180,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -205,13 +188,11 @@ 1 55 1055 0 0 -10 0 990 1 55 15 1055 1015 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -219,11 +200,8 @@ 1 1 1 -{% endif -%} -{% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -239,7 +217,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -252,7 +229,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -263,7 +239,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -271,7 +246,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -287,7 +261,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -300,7 +273,6 @@ 9 9 \N 10 10 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -311,7 +283,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -320,7 +291,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -336,7 +306,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -349,7 +318,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -360,7 +328,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -368,7 +335,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -384,7 +350,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -397,7 +362,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -408,7 +372,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -418,7 +381,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -427,13 +389,11 @@ 1 55 1055 1 55 15 1055 1015 \N \N -10 \N 990 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -441,6 +401,4 @@ 1 1 1 -{% endif -%} -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 98bfd9d9b2b..7b994c2d11f 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -50,7 +50,6 @@ ALL LEFT | bs = {{ block_size }} 14 14 val9 0 14 14 val9 0 ALL RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -64,7 +63,6 @@ ALL RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -85,7 +83,6 @@ ALL LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -99,7 +96,6 @@ ALL RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 -{% endif -%} ANY INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -137,7 +133,6 @@ ANY LEFT | bs = {{ block_size }} 13 13 val13 0 14 14 val9 0 ANY RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -150,7 +145,6 @@ ANY RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | bs = {{ block_size }} | copmosite key @@ -170,7 +164,6 @@ ANY LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -183,7 +176,6 @@ ANY RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 1 \N 1 val3 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 -{% endif -%} {% endfor -%} ALL INNER | join_use_nulls = 1 4 4 0 0 @@ -219,7 +211,6 @@ ALL LEFT | join_use_nulls = 1 14 14 val9 0 14 14 val9 0 ALL RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -233,7 +224,6 @@ ALL RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -254,7 +244,6 @@ ALL LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 @@ -268,7 +257,6 @@ ALL RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} ANY INNER | join_use_nulls = 1 4 4 0 0 5 5 0 0 @@ -296,7 +284,6 @@ ANY LEFT | join_use_nulls = 1 13 13 val13 0 14 14 val9 0 ANY RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -309,7 +296,6 @@ ANY RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | join_use_nulls = 1 | copmosite key @@ -329,7 +315,6 @@ ANY LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 \N \N \N 1 1 1 \N val7 @@ -342,5 +327,4 @@ ANY RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2cc6c6e85d6..6b5b5bfd298 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -108,7 +108,6 @@ t1 ALL LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -161,7 +160,6 @@ t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val28 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 2 2 5 5 @@ -177,7 +175,6 @@ t1 ANY LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -188,9 +185,7 @@ t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val27 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 2 2 5 5 @@ -243,9 +238,7 @@ t1 ALL FULL JOIN t2 | bs = {{ block_size }} 2 2 5 5 2 2 5 5 3 3 4 4 -{% endif -%} t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 5 @@ -298,7 +291,6 @@ t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} 2 5 5 2 5 5 3 4 4 -{% endif -%} t1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -315,7 +307,6 @@ t1 ALL LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -326,7 +317,6 @@ t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -341,7 +331,6 @@ t1 ANY LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -352,9 +341,7 @@ t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 5 0 \N 0 5 0 \N 0 5 @@ -372,9 +359,7 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 -{% endif -%} t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 0 @@ -392,7 +377,6 @@ t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 0 5 \N 0 5 \N 0 5 -{% endif -%} tn1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -409,7 +393,6 @@ tn1 ALL LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -420,7 +403,6 @@ tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -435,7 +417,6 @@ tn1 ANY LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -446,9 +427,7 @@ tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -466,9 +445,7 @@ tn1 ALL FULL JOIN t2 | bs = {{ block_size }} \N 2 0 5 \N 2 0 5 \N 2 0 5 -{% endif -%} tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 0 5 @@ -486,7 +463,6 @@ tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 -{% endif -%} tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -503,7 +479,6 @@ tn1 ALL LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -514,7 +489,6 @@ tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -529,7 +503,6 @@ tn1 ANY LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -540,9 +513,7 @@ tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -560,9 +531,7 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 -{% endif -%} tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 3 4 4 @@ -580,6 +549,5 @@ tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 -{% endif -%} {% endfor -%} {% endfor -%} From 5c44e6a562f23a388b9895088320286b1bf7b598 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 6 May 2023 16:26:40 +0800 Subject: [PATCH 117/129] triger ci --- src/Interpreters/GraceHashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index ee369d4b195..bdfd63602ec 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -501,7 +501,7 @@ public: non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); } } - // For right/full join, non_joined_blocks is not null. + // For right/full join, non_joined_blocks is not null // For inner/left join, non_joined_blocks is null, and return a empty block directly. std::lock_guard lock(mutex); if (non_joined_blocks) From 89dd538beac9f1b690cf772764939318fc6cc47a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 09:29:29 +0800 Subject: [PATCH 118/129] update --- src/Interpreters/GraceHashJoin.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index bdfd63602ec..7d924bb3a02 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -629,6 +629,16 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; + if (hash_join) + { + auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); + for (auto & block : right_blocks) + { + Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size()); + flushBlocksToBuckets(blocks, buckets, bucket_idx); + } + } + hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) From 8efec9bccab4097574985030fa5aa112bc357242 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 09:59:56 +0800 Subject: [PATCH 119/129] add locks for getNonJoinedBlocks --- src/Interpreters/GraceHashJoin.cpp | 13 ++++++++++++- src/Interpreters/GraceHashJoin.h | 2 ++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 7d924bb3a02..0e839e538fd 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -524,7 +525,17 @@ private: IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + std::lock_guard lock(hash_join_mutex); + if (!non_joined_block_stream) + { + if (last_hash_join_for_non_joined && last_hash_join_for_non_joined != hash_join) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "getNonJoinedBlocks should be called once"); + } + non_joined_block_stream = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + last_hash_join_for_non_joined = hash_join; + } + return non_joined_block_stream; } class GraceHashJoin::DelayedBlocks : public IBlocksStream diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index bc5f894bc8c..0ff3bc3f609 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -145,6 +145,8 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; + mutable IBlocksStreamPtr non_joined_block_stream = nullptr; + mutable InMemoryJoinPtr last_hash_join_for_non_joined = nullptr; }; } From d5efc0e6884b39ac77f31e1acff6a8eb370261ce Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 11:35:48 +0800 Subject: [PATCH 120/129] update --- src/Interpreters/GraceHashJoin.cpp | 15 +++------------ src/Interpreters/GraceHashJoin.h | 2 -- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 0e839e538fd..06a4cfaa76a 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -522,20 +522,11 @@ private: // This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in // DelayedBlocks. +// There is a finished counter in JoiningTransform, only the last JoiningTransform could call this function. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - std::lock_guard lock(hash_join_mutex); - if (!non_joined_block_stream) - { - if (last_hash_join_for_non_joined && last_hash_join_for_non_joined != hash_join) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "getNonJoinedBlocks should be called once"); - } - non_joined_block_stream = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); - last_hash_join_for_non_joined = hash_join; - } - return non_joined_block_stream; + return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -582,7 +573,7 @@ public: is_left_reader_finished = true; } } - if (is_left_reader_finished || !block) + if (is_left_reader_finished && !block) { return non_joined_blocks_iter.next(); } diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index 0ff3bc3f609..bc5f894bc8c 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -145,8 +145,6 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; - mutable IBlocksStreamPtr non_joined_block_stream = nullptr; - mutable InMemoryJoinPtr last_hash_join_for_non_joined = nullptr; }; } From 80af345ea6b8fa3e10ce84578fca2aaf76808ee1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 14:09:41 +0800 Subject: [PATCH 121/129] update --- src/Interpreters/GraceHashJoin.cpp | 3 ++- .../0_stateless/01721_join_implicit_cast_long.reference.j2 | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 06a4cfaa76a..55ad0971a8f 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -571,9 +571,10 @@ public: if (!block) { is_left_reader_finished = true; + return non_joined_blocks_iter.next(); } } - if (is_left_reader_finished && !block) + else { return non_joined_blocks_iter.next(); } diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index d40fc7ec864..ae43aa7195c 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -200,6 +200,7 @@ 1 1 1 +{% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = -4 \N 196 @@ -401,4 +402,5 @@ 1 1 1 +{% endif -%} {% endfor -%} From 29ade233979aab464dcc4d8141b2a51166a7bcb5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 15:18:32 +0800 Subject: [PATCH 122/129] fixed: return invalid mismatch rows on full/right join --- src/Interpreters/GraceHashJoin.cpp | 80 ++++++++++-------------------- src/Interpreters/GraceHashJoin.h | 1 - 2 files changed, 25 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 55ad0971a8f..2e622a4c18e 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -13,7 +12,6 @@ #include #include #include -#include "GraceHashJoin.h" #include #include @@ -474,59 +472,13 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -// Add lock for reading non-joined blocks, it's thread safe. -class GraceHashJoin::NonJoinedBlocksStream : public IBlocksStream -{ -public: - explicit NonJoinedBlocksStream( - InMemoryJoinPtr hash_join_, - const Block & left_sample_block_, - const Block & result_sample_block_, - size_t max_block_size_) - : hash_join(hash_join_) - , left_sample_block(left_sample_block_) - , result_sample_block(result_sample_block_) - , max_block_size(max_block_size_) - { - } - - Block nextImpl() override - { - // initialize non_joined_blocks lazily. - if (!has_init_non_joined_blocks) [[unlikely]] - { - std::lock_guard lock(mutex); - if (!has_init_non_joined_blocks) - { - has_init_non_joined_blocks = true; - non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); - } - } - // For right/full join, non_joined_blocks is not null - // For inner/left join, non_joined_blocks is null, and return a empty block directly. - std::lock_guard lock(mutex); - if (non_joined_blocks) - return non_joined_blocks->next(); - return {}; - } - -private: - InMemoryJoinPtr hash_join; - Block left_sample_block; - Block result_sample_block; - size_t max_block_size; - bool has_init_non_joined_blocks = false; - IBlocksStreamPtr non_joined_blocks = nullptr; - std::mutex mutex; -}; - // This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in // DelayedBlocks. // There is a finished counter in JoiningTransform, only the last JoiningTransform could call this function. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -547,7 +499,9 @@ public: , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) - , non_joined_blocks_iter(hash_join, left_sample_block_, result_sample_block_, max_block_size_) + , left_sample_block(left_sample_block_) + , result_sample_block(result_sample_block_) + , max_block_size(max_block_size_) { } @@ -557,6 +511,12 @@ public: // changed inside. Block nextImpl() override { + // there is data race case wihthout this lock: + // 1. thread 1 read the last block from left_reader, but not finish the join + // 2. thread 2 try to read from non-joined blocks. Since thread 1 has not finished, + // the used flags in the hash_join is incomplete, then thread 2 return invalid mismatch rows. + std::lock_guard lock(non_joined_blocks_mutex); + Block block; size_t num_buckets = buckets.size(); size_t current_idx = buckets[current_bucket]->idx; @@ -571,12 +531,18 @@ public: if (!block) { is_left_reader_finished = true; - return non_joined_blocks_iter.next(); + non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); } } - else + + if (is_left_reader_finished) { - return non_joined_blocks_iter.next(); + if (non_joined_blocks) + { + return non_joined_blocks->next(); + } + else + return {}; } // block comes from left_reader, need to join with right table to get the result. @@ -619,8 +585,12 @@ public: Names left_key_names; Names right_key_names; - std::atomic is_left_reader_finished = false; - NonJoinedBlocksStream non_joined_blocks_iter; + Block left_sample_block; + Block result_sample_block; + size_t max_block_size = 0; + bool is_left_reader_finished = false; + IBlocksStreamPtr non_joined_blocks = nullptr; + std::mutex non_joined_blocks_mutex; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index bc5f894bc8c..eb39ee09208 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -43,7 +43,6 @@ class HashJoin; class GraceHashJoin final : public IJoin { class FileBucket; - class NonJoinedBlocksStream; class DelayedBlocks; using InMemoryJoin = HashJoin; From 603c024eb0615392277297273d62e9b790237416 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 9 May 2023 11:17:09 +0800 Subject: [PATCH 123/129] ensure only the last processor could access non-joined blocks --- src/Interpreters/GraceHashJoin.cpp | 67 +++--------------- .../Transforms/JoiningTransform.cpp | 69 +++++++++++++++++-- src/Processors/Transforms/JoiningTransform.h | 24 ++++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...274_full_sort_join_nodistinct.reference.j2 | 20 ++++++ 5 files changed, 115 insertions(+), 67 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 2e622a4c18e..ad4457f924e 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -472,9 +472,9 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -// This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in -// DelayedBlocks. -// There is a finished counter in JoiningTransform, only the last JoiningTransform could call this function. +// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, +// only the last processor could call this function to ensure all used flags have been inserted. +// To support delayed stream mode, need to keep the hash join before next getDelayedBlocks call. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { @@ -489,60 +489,30 @@ public: Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, - const Names & right_key_names_, - const Block & left_sample_block_, - const Block & result_sample_block_, - size_t max_block_size_) + const Names & right_key_names_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) - , left_sample_block(left_sample_block_) - , result_sample_block(result_sample_block_) - , max_block_size(max_block_size_) { } - // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransforms, need locks for - // - reading from left_reader. left_reader.read() has had a lock inside. - // - reading non-joined blocks from hash_join. Since iterate on non-joined blocks will has state - // changed inside. Block nextImpl() override { - // there is data race case wihthout this lock: - // 1. thread 1 read the last block from left_reader, but not finish the join - // 2. thread 2 try to read from non-joined blocks. Since thread 1 has not finished, - // the used flags in the hash_join is incomplete, then thread 2 return invalid mismatch rows. - std::lock_guard lock(non_joined_blocks_mutex); - Block block; size_t num_buckets = buckets.size(); size_t current_idx = buckets[current_bucket]->idx; do { - // When left reader finish, return non-joined blocks. - // empty block means the end of this stream. - if (!is_left_reader_finished) + // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform. + // There is a lock inside left_reader.read(). + block = left_reader.read(); + if (!block) { - block = left_reader.read(); - if (!block) - { - is_left_reader_finished = true; - non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); - } - } - - if (is_left_reader_finished) - { - if (non_joined_blocks) - { - return non_joined_blocks->next(); - } - else - return {}; + return {}; } // block comes from left_reader, need to join with right table to get the result. @@ -585,12 +555,6 @@ public: Names left_key_names; Names right_key_names; - Block left_sample_block; - Block result_sample_block; - size_t max_block_size = 0; - bool is_left_reader_finished = false; - IBlocksStreamPtr non_joined_blocks = nullptr; - std::mutex non_joined_blocks_mutex; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() @@ -636,18 +600,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() LOG_TRACE(log, "Loaded bucket {} with {}(/{}) rows", bucket_idx, hash_join->getTotalRowCount(), num_rows); - auto result_sample_block = left_sample_block; - ExtraBlockPtr tmp; - hash_join->joinBlock(result_sample_block, tmp); - return std::make_unique( - current_bucket->idx, - buckets, - hash_join, - left_key_names, - right_key_names, - left_sample_block, - result_sample_block, - max_block_size); + return std::make_unique(current_bucket->idx, buckets, hash_join, left_key_names, right_key_names); } LOG_TRACE(log, "Finished loading all {} buckets", buckets.size()); diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index b638a5582c5..d80d6b8e387 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -311,8 +311,16 @@ void FillingRightJoinSideTransform::work() } -DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block output_header) - : IProcessor(InputPorts{Block()}, OutputPorts{output_header}) +DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_) + : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) + , left_header(left_header_) + , output_header(output_header_) + , max_block_size(max_block_size_) + , join(join_) { } @@ -366,6 +374,7 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!data.chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); + } else { @@ -386,12 +395,24 @@ void DelayedJoinedBlocksWorkerTransform::work() { if (!task) return; + Block block; + if (!left_delayed_stream_finished) + { + block = task->delayed_blocks->next(); - Block block = task->delayed_blocks->next(); - + if (!block) + { + left_delayed_stream_finished = true; + block = nextNonJoinedBlock(); + } + } + else + { + block = nextNonJoinedBlock(); + } if (!block) { - task.reset(); + resetTask(); return; } @@ -400,6 +421,38 @@ void DelayedJoinedBlocksWorkerTransform::work() output_chunk.setColumns(block.getColumns(), rows); } +void DelayedJoinedBlocksWorkerTransform::resetTask() +{ + task.reset(); + left_delayed_stream_finished = false; + setup_non_joined_stream = false; + non_joined_delay_stream = nullptr; +} + +Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() +{ + if (!setup_non_joined_stream) + { + setup_non_joined_stream = true; + // Before read from non-joined stream, all blocks in left file reader must have been joined. + // For example, in HashJoin, it may return invalid mismatch rows from non-joined stream before + // the all blocks in left file reader have been finished, since the used flags are incomplete. + // To make only one processor could read from non-joined stream seems be a easy way. + if (task && task->left_delayed_stream_finish_counter->isLast()) + { + if (!non_joined_delay_stream) + { + non_joined_delay_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + } + } + } + if (non_joined_delay_stream) + { + return non_joined_delay_stream->next(); + } + return {}; +} + DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, JoinPtr join_) : IProcessor(InputPorts{}, OutputPorts(num_streams, Block())) , join(std::move(join_)) @@ -451,10 +504,14 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { + // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform + // could read non-joined blocks from the join. + auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); + auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); + chunk.setChunkInfo(task); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e7edff40c56..e9bd79bf623 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -116,9 +116,14 @@ class DelayedBlocksTask : public ChunkInfo public: explicit DelayedBlocksTask() : finished(true) {} - explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_) : delayed_blocks(std::move(delayed_blocks_)) {} + explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) + : delayed_blocks(std::move(delayed_blocks_)) + , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) + { + } IBlocksStreamPtr delayed_blocks = nullptr; + JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; bool finished = false; }; @@ -147,7 +152,11 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: - explicit DelayedJoinedBlocksWorkerTransform(Block output_header); + explicit DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -155,10 +164,19 @@ public: void work() override; private: + Block left_header; + Block output_header; + size_t max_block_size; + JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - bool finished = false; + bool left_delayed_stream_finished = false; + bool setup_non_joined_stream = false; + IBlocksStreamPtr non_joined_delay_stream = nullptr; + + void resetTask(); + Block nextNonJoinedBlock(); }; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index a4edf107b2f..764997e7b7e 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(joined_header); + auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 6b5b5bfd298..2dc7fe33423 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -360,6 +360,7 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 3 3 4 4 t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} +{% if join_algorithm == 'full_sorting_merge' -%} 1 4 5 1 4 5 2 5 0 @@ -377,6 +378,25 @@ t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 0 5 \N 0 5 \N 0 5 +{% else -%} +0 0 5 +0 0 5 +0 0 5 +0 0 5 +0 0 5 +0 0 5 +0 0 5 +1 4 5 +1 4 5 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +3 4 4 +{% endif -%} tn1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 From 258239927109d808276f475d02c37a245c84f944 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 9 May 2023 17:42:07 +0800 Subject: [PATCH 124/129] fixed: a in-memory bucket contains rows of other buckets --- src/Interpreters/GraceHashJoin.cpp | 22 ++++++++++--------- .../Transforms/JoiningTransform.cpp | 11 +++++----- src/Processors/Transforms/JoiningTransform.h | 2 +- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index ad4457f924e..129d58a5436 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -566,16 +566,6 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; - if (hash_join) - { - auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); - for (auto & block : right_blocks) - { - Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size()); - flushBlocksToBuckets(blocks, buckets, bucket_idx); - } - } - hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) @@ -639,6 +629,18 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) if (!hash_join) hash_join = makeInMemoryJoin(); + // buckets size has been changed in other threads. Need to scatter current_block again. + // rehash could only happen under hash_join_mutex's scope. + auto current_buckets = getCurrentBuckets(); + if (buckets_snapshot.size() != current_buckets.size()) + { + LOG_TRACE(log, "mismatch buckets size. previous:{}, current:{}", buckets_snapshot.size(), getCurrentBuckets().size()); + Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, current_block, current_buckets.size()); + flushBlocksToBuckets(blocks, current_buckets, bucket_index); + current_block = std::move(blocks[bucket_index]); + if (!current_block.rows()) + return; + } hash_join->addJoinedBlock(current_block, /* check_limits = */ false); if (!hasMemoryOverflow(hash_join)) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index d80d6b8e387..d4ea601209f 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -189,7 +189,6 @@ void JoiningTransform::transform(Chunk & chunk) } else block = readExecute(chunk); - auto num_rows = block.rows(); chunk.setColumns(block.getColumns(), num_rows); } @@ -426,7 +425,7 @@ void DelayedJoinedBlocksWorkerTransform::resetTask() task.reset(); left_delayed_stream_finished = false; setup_non_joined_stream = false; - non_joined_delay_stream = nullptr; + non_joined_delayed_stream = nullptr; } Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() @@ -440,15 +439,15 @@ Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() // To make only one processor could read from non-joined stream seems be a easy way. if (task && task->left_delayed_stream_finish_counter->isLast()) { - if (!non_joined_delay_stream) + if (!non_joined_delayed_stream) { - non_joined_delay_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); } } } - if (non_joined_delay_stream) + if (non_joined_delayed_stream) { - return non_joined_delay_stream->next(); + return non_joined_delayed_stream->next(); } return {}; } diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e9bd79bf623..17222f0e514 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -173,7 +173,7 @@ private: bool left_delayed_stream_finished = false; bool setup_non_joined_stream = false; - IBlocksStreamPtr non_joined_delay_stream = nullptr; + IBlocksStreamPtr non_joined_delayed_stream = nullptr; void resetTask(); Block nextNonJoinedBlock(); From 983514f9461862b6d999455863f3178dd09ab2a1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 10 May 2023 16:00:41 +0800 Subject: [PATCH 125/129] update tests for using clause --- ...274_full_sort_join_nodistinct.reference.j2 | 22 +++---------------- .../02274_full_sort_join_nodistinct.sql.j2 | 3 ++- 2 files changed, 5 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2dc7fe33423..2b8ca580f50 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -359,8 +359,8 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 +{% if join_algorithm != 'grace_hash' -%} t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} -{% if join_algorithm == 'full_sorting_merge' -%} 1 4 5 1 4 5 2 5 0 @@ -378,24 +378,6 @@ t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 0 5 \N 0 5 \N 0 5 -{% else -%} -0 0 5 -0 0 5 -0 0 5 -0 0 5 -0 0 5 -0 0 5 -0 0 5 -1 4 5 -1 4 5 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -3 4 4 {% endif -%} tn1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 @@ -551,6 +533,7 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 +{% if join_algorithm != 'grace_hash' -%} tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 @@ -569,5 +552,6 @@ tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 +{% endif -%} {% endfor -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 6bf634ca1f5..a1f7b133cf6 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -46,9 +46,10 @@ SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT J SELECT '{{ t1 }} ALL FULL JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} FULL JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} +{% if join_algorithm == 'full_sorting_merge' or t2 != 'tn2' -%} SELECT '{{ t1 }} ALL FULL JOIN USING {{ t2 }} | bs = {{ block_size }}'; SELECT key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ALL FULL JOIN {{ t2 }} AS t2 USING (key) ORDER BY key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} - +{% endif -%} {% endfor -%} {% endfor -%} SET max_bytes_in_join = 0; From 35d83887057e245dc038e918913be45fa3ca66ae Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 May 2023 09:38:21 +0800 Subject: [PATCH 126/129] update --- src/Interpreters/GraceHashJoin.cpp | 13 +++++++++---- src/Processors/Transforms/JoiningTransform.cpp | 2 +- src/Processors/Transforms/JoiningTransform.h | 1 + 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 129d58a5436..4a4c69ff473 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -471,10 +471,15 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } - -// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, -// only the last processor could call this function to ensure all used flags have been inserted. -// To support delayed stream mode, need to keep the hash join before next getDelayedBlocks call. +/// Each bucket are handled by the following steps +/// 1. build hash_join by the right side blocks. +/// 2. join left side with the hash_join, +/// 3. read right non-joined blocks from hash_join. +/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted. +/// +/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, +/// only one processor could take the non-joined blocks from right stream, and ensure all rows from +/// left stream have been emitted before this. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index d4ea601209f..256ef66a27d 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -504,7 +504,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform - // could read non-joined blocks from the join. + // could read right non-joined blocks from the join. auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 17222f0e514..3577906b26a 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -171,6 +171,7 @@ private: DelayedBlocksTaskPtr task; Chunk output_chunk; + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left bool left_delayed_stream_finished = false; bool setup_non_joined_stream = false; IBlocksStreamPtr non_joined_delayed_stream = nullptr; From cc93f9ab52669dda67dce644241c4f7e88b94216 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 May 2023 09:06:33 +0800 Subject: [PATCH 127/129] update tests for flaky test --- docs/en/operations/settings/settings.md | 2 ++ tests/queries/0_stateless/02273_full_sort_join.reference.j2 | 2 +- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 2 +- .../0_stateless/02274_full_sort_join_nodistinct.reference.j2 | 2 +- .../queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 | 2 +- 5 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 791186dd9c1..2239084a429 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -452,6 +452,8 @@ Possible values: The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. + Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`. + - hash [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 7b994c2d11f..0af4158e971 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -1,7 +1,7 @@ {% set table_size = 15 -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} -- {{ join_algorithm }} -- -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} ALL INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index b371d9544da..6500306356c 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -36,7 +36,7 @@ SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} {% for kind in ['ALL', 'ANY'] -%} SET max_block_size = {{ block_size }}; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2b8ca580f50..df968e86e8d 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -1,6 +1,6 @@ {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} --- {{ join_algorithm }} --- -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} t1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index a1f7b133cf6..f8eb4b1a53e 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -26,7 +26,7 @@ SET join_algorithm = '{{ join_algorithm }}'; SELECT '--- {{ join_algorithm }} ---'; -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} SET max_block_size = {{ block_size }}; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} From c30658a9ed4dd8f144e9da3b5696c7921f5aa5cf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 22 May 2023 08:57:03 +0200 Subject: [PATCH 128/129] Fix hashed/sparse_hashed dictionaries max_load_factor upper range Previously due to comparison of floats with doubles, it was incorrectly works for the upper range: (lldb) p (float)0.99 > (float)0.99 (bool) $0 = false (lldb) p (float)0.99 > (double)0.99 (bool) $1 = true This should also fix performance tests errors on CI: clickhouse_driver.errors.ServerException: Code: 36. DB::Exception: default.simple_key_HASHED_dictionary_l0_99: max_load_factor parameter should be within [0.5, 0.99], got 0.99. Stack trace: Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 83d051df5e9..b3d8ca8f5ce 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -1163,7 +1163,7 @@ void registerDictionaryHashed(DictionaryFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS,"{}: SHARD_LOAD_QUEUE_BACKLOG parameter should be greater then zero", full_name); float max_load_factor = static_cast(config.getDouble(config_prefix + dictionary_layout_prefix + ".max_load_factor", 0.5)); - if (max_load_factor < 0.5 || max_load_factor > 0.99) + if (max_load_factor < 0.5f || max_load_factor > 0.99f) throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: max_load_factor parameter should be within [0.5, 0.99], got {}", full_name, max_load_factor); HashedDictionaryConfiguration configuration{ From a257ff6cf3ba1124e861452e4b3b52ada3ea2d5c Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 22 May 2023 10:41:22 +0200 Subject: [PATCH 129/129] address comment --- src/Common/OptimizedRegularExpression.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 92b3ad32ecd..f2fe922ef19 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -68,7 +68,7 @@ const char * analyzeImpl( bool has_alternative_on_depth_0 = false; bool has_case_insensitive_flag = false; - /// Substring with a position. + /// Substring with is_prefix. using Substring = std::pair; using Substrings = std::vector;