From f0bfdb6b029748a486e5f683171f135d6a5dd957 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 22 Feb 2023 16:24:02 +0000 Subject: [PATCH 01/72] Refactor Query Tree visitor --- src/Analyzer/InDepthQueryTreeVisitor.h | 65 +++++++++++++++++++++++ src/Analyzer/Passes/CountDistinctPass.cpp | 34 ++++++++---- 2 files changed, 88 insertions(+), 11 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index 1cc48fb1e53..ee321842ffa 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -235,4 +235,69 @@ public: template using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor; +template +class QueryTreeVisitor +{ +public: + explicit QueryTreeVisitor(ContextPtr context_) + : current_context(std::move(context_)) + {} + + bool needApply(QueryTreeNodePtr & node) + { + return getImpl().needApply(node); + } + + void visit(QueryTreeNodePtr & node) + { + auto current_scope_context_ptr = current_context; + SCOPE_EXIT( + current_context = std::move(current_scope_context_ptr); + ); + + if (auto * query_node = node->template as()) + current_context = query_node->getContext(); + else if (auto * union_node = node->template as()) + current_context = union_node->getContext(); + + if (!TOP_TO_BOTTOM) + visitChildren(node); + + if (needApply(node)) + getImpl().apply(node); + + if (TOP_TO_BOTTOM) + visitChildren(node); + } + + const ContextPtr & getContext() const + { + return current_context; + } + + const Settings & getSettings() const + { + return current_context->getSettingsRef(); + } +private: + + Impl & getImpl() + { + return *static_cast(this); + } + + void visitChildren(QueryTreeNodePtr & node) + { + for (auto & child : node->getChildren()) + { + if (child) + visit(child); + } + } + + static constexpr bool TOP_TO_BOTTOM = Impl::TOP_TO_BOTTOM; + + ContextPtr current_context; +}; + } diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 945295f5cbc..38f7d07d052 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -16,16 +16,17 @@ namespace DB namespace { -class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContext +class CountDistinctVisitor : public QueryTreeVisitor { public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; + using QueryTreeVisitor::QueryTreeVisitor; - void visitImpl(QueryTreeNodePtr & node) + static constexpr bool TOP_TO_BOTTOM = true; + + bool needApply(QueryTreeNodePtr & node) { if (!getSettings().count_distinct_optimization) - return; + return false; auto * query_node = node->as(); @@ -33,32 +34,43 @@ public: if (!query_node || (query_node->hasWith() || query_node->hasPrewhere() || query_node->hasWhere() || query_node->hasGroupBy() || query_node->hasHaving() || query_node->hasWindow() || query_node->hasOrderBy() || query_node->hasLimitByLimit() || query_node->hasLimitByOffset() || query_node->hasLimitBy() || query_node->hasLimit() || query_node->hasOffset())) - return; + return false; /// Check that query has only single table expression auto join_tree_node_type = query_node->getJoinTree()->getNodeType(); if (join_tree_node_type == QueryTreeNodeType::JOIN || join_tree_node_type == QueryTreeNodeType::ARRAY_JOIN) - return; + return false; /// Check that query has only single node in projection auto & projection_nodes = query_node->getProjection().getNodes(); if (projection_nodes.size() != 1) - return; + return false; /// Check that query single projection node is `countDistinct` function auto & projection_node = projection_nodes[0]; auto * function_node = projection_node->as(); if (!function_node) - return; + return false; auto lower_function_name = Poco::toLower(function_node->getFunctionName()); if (lower_function_name != "countdistinct" && lower_function_name != "uniqexact") - return; + return false; /// Check that `countDistinct` function has single COLUMN argument auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) - return; + return false; + + return true; + } + + void apply(QueryTreeNodePtr & node) + { + auto * query_node = node->as(); + auto & projection_nodes = query_node->getProjection().getNodes(); + auto * function_node = projection_nodes[0]->as(); + + auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); From 5c34ee3019199a7e1d24730684c9c84e6c8e0615 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 9 May 2023 15:14:49 +0000 Subject: [PATCH 02/72] Skip unresolved table function arguments --- src/Analyzer/InDepthQueryTreeVisitor.h | 15 ++++++++++++++- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Analyzer/TableFunctionNode.cpp | 3 ++- src/Analyzer/TableFunctionNode.h | 8 +++++++- 4 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index ee321842ffa..be3a760d4e6 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -248,6 +249,16 @@ public: return getImpl().needApply(node); } + bool shouldSkipSubtree(QueryTreeNodePtr & parent, size_t subtree_index) + { + if (auto * table_function_node = parent->as()) + { + const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes(); + return std::find(unresolved_indexes.begin(), unresolved_indexes.end(), subtree_index) != unresolved_indexes.end(); + } + return false; + } + void visit(QueryTreeNodePtr & node) { auto current_scope_context_ptr = current_context; @@ -288,10 +299,12 @@ private: void visitChildren(QueryTreeNodePtr & node) { + size_t index = 0; for (auto & child : node->getChildren()) { - if (child) + if (child && !shouldSkipSubtree(node, index)) visit(child); + ++index; } } diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7ab0261850b..aaea81dcada 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6356,7 +6356,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, table_function_ptr->parseArguments(table_function_ast, scope_context); auto table_function_storage = table_function_ptr->execute(table_function_ast, scope_context, table_function_ptr->getName()); - table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context); + table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context, std::move(skip_analysis_arguments_indexes)); } /// Resolve array join node in scope diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index c130503d660..30644ad4ec4 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -27,12 +27,13 @@ TableFunctionNode::TableFunctionNode(String table_function_name_) children[arguments_child_index] = std::make_shared(); } -void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context) +void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector unresolved_arguments_indexes_) { table_function = std::move(table_function_value); storage = std::move(storage_value); storage_id = storage->getStorageID(); storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); + unresolved_arguments_indexes = std::move(unresolved_arguments_indexes_); } const StorageID & TableFunctionNode::getStorageID() const diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 7786ba62205..69237ac8416 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -98,7 +98,7 @@ public: } /// Resolve table function with table function, storage and context - void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context); + void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector unresolved_arguments_indexes_); /// Get storage id, throws exception if function node is not resolved const StorageID & getStorageID() const; @@ -106,6 +106,11 @@ public: /// Get storage snapshot, throws exception if function node is not resolved const StorageSnapshotPtr & getStorageSnapshot() const; + const std::vector & getUnresolvedArgumentIndexes() const + { + return unresolved_arguments_indexes; + } + /// Return true if table function node has table expression modifiers, false otherwise bool hasTableExpressionModifiers() const { @@ -164,6 +169,7 @@ private: StoragePtr storage; StorageID storage_id; StorageSnapshotPtr storage_snapshot; + std::vector unresolved_arguments_indexes; std::optional table_expression_modifiers; SettingsChanges settings_changes; From 8b1cd9fcec4408933d537bd1b74a382884d7b52f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Jun 2023 02:27:10 +0200 Subject: [PATCH 03/72] Remove metadata_cache from config --- programs/server/config.xml | 6 ------ 1 file changed, 6 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index d18b4cb2ac9..f52241ff44d 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1550,12 +1550,6 @@ - - - + --96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml index 5f7769f7403..830c75f7378 100644 --- a/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml +++ b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml @@ -1,3 +1,7 @@ - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + diff --git a/tests/integration/test_config_decryption/configs/config_subnodes.xml b/tests/integration/test_config_decryption/configs/config_subnodes.xml index b0e519ff546..8213270f747 100644 --- a/tests/integration/test_config_decryption/configs/config_subnodes.xml +++ b/tests/integration/test_config_decryption/configs/config_subnodes.xml @@ -1,10 +1,14 @@ + 00112233445566778899aabbccddeeff - + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + diff --git a/tests/integration/test_config_decryption/configs/config_wrong_method.xml b/tests/integration/test_config_decryption/configs/config_wrong_method.xml index b452ce6374c..b96c13d5105 100644 --- a/tests/integration/test_config_decryption/configs/config_wrong_method.xml +++ b/tests/integration/test_config_decryption/configs/config_wrong_method.xml @@ -1,4 +1,5 @@ + 00112233445566778899aabbccddeeff @@ -7,6 +8,8 @@ 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index b148f9a051a..c6987d12324 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -15,7 +15,7 @@ def start_clickhouse(config, err_msg): def test_wrong_method(): start_clickhouse( - "configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG" + "configs/config_wrong_method.xml", "Unknown encryption method. Got WRONG" ) From dccbe875d247818a17e999ceab5e062537169f80 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 26 Jul 2023 15:37:58 +0200 Subject: [PATCH 24/72] check if storage shutdown before we operate MergeTreeDeduplicationLog --- .../MergeTree/MergeTreeDeduplicationLog.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 09a04f13fc7..53481ab06a0 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,6 +10,8 @@ #include #include +#include + namespace DB { @@ -231,6 +233,12 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: return std::make_pair(info, false); } + if (stopped) + { + LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we add this part."); + return {}; + } + chassert(current_writer != nullptr); /// Create new record @@ -261,6 +269,12 @@ void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_inf if (deduplication_window == 0) return; + if (stopped) + { + LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we drop this part."); + return; + } + chassert(current_writer != nullptr); for (auto itr = deduplication_map.begin(); itr != deduplication_map.end(); /* no increment here, we erasing from map */) From 338188ae7f1ccdb399671cbfae584ff79705097a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 26 Jul 2023 14:10:27 +0000 Subject: [PATCH 25/72] fix test --- tests/queries/0_stateless/02791_remote_paths_refcount.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql index e64df599d32..180601738ad 100644 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.sql +++ b/tests/queries/0_stateless/02791_remote_paths_refcount.sql @@ -2,6 +2,10 @@ DROP TABLE IF EXISTS t_refcount SYNC; +-- Names of parts (on which this test depends) +-- can differ in case of fault injection. +SET insert_keeper_fault_injection_probability = 0.0; + CREATE TABLE t_refcount (id UInt64, v UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') ORDER BY id PARTITION BY id % 2 From 89f2e8cdea8d7f32be735cd86326d1cbed24e158 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 26 Jul 2023 23:02:57 +0800 Subject: [PATCH 26/72] Fix S3 table function does not work for pre-signed URL --- src/TableFunctions/TableFunctionS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index c8cc0cddd30..3637b3e9eb2 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -160,7 +160,7 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context configuration.keys = {configuration.url.key}; if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url.uri.getPath(), true); + configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(configuration.url.uri.getPath()).getPath(), true); } void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) From 2479f1352a62adebdc460dbfde4510ad25fc7184 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 17:11:28 +0200 Subject: [PATCH 27/72] fix deadlocks in StorageTableFunctionProxy --- src/Storages/StorageTableFunction.h | 2 +- .../02828_create_as_table_function_rename.reference | 1 + .../0_stateless/02828_create_as_table_function_rename.sql | 7 +++++++ 3 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02828_create_as_table_function_rename.reference create mode 100644 tests/queries/0_stateless/02828_create_as_table_function_rename.sql diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 26cbe1f0233..3939483495e 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -156,7 +156,7 @@ public: void checkTableCanBeDropped() const override {} private: - mutable std::mutex nested_mutex; + mutable std::recursive_mutex nested_mutex; mutable GetNestedStorageFunc get_nested; mutable StoragePtr nested; const bool add_conversion; diff --git a/tests/queries/0_stateless/02828_create_as_table_function_rename.reference b/tests/queries/0_stateless/02828_create_as_table_function_rename.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02828_create_as_table_function_rename.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02828_create_as_table_function_rename.sql b/tests/queries/0_stateless/02828_create_as_table_function_rename.sql new file mode 100644 index 00000000000..7e24e485fb9 --- /dev/null +++ b/tests/queries/0_stateless/02828_create_as_table_function_rename.sql @@ -0,0 +1,7 @@ + +drop table if exists t1; +create table t1 as remote('localhost', 'system.one'); +rename table t1 to t2; +select * from t2; +rename table t2 to t1; +drop table t1; From 74f3e76b182411fee1d49e74aa5040cd4a378967 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 17:15:28 +0200 Subject: [PATCH 28/72] fix build with clang-15 --- src/Common/SystemLogBase.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 3 +++ src/Storages/HDFS/StorageHDFS.h | 2 ++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index ed5ffd78a7b..3d68fe63227 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -185,7 +185,7 @@ void SystemLogQueue::confirm(uint64_t to_flush_end) } template -SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) +typename SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) { std::unique_lock lock(mutex); flush_event.wait_for(lock, diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 6a68d0f21f7..783c71448fc 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -360,6 +360,9 @@ private: struct DownloadInfo { + DownloadInfo(const CacheMetadata::Key & key_, const size_t & offset_, const std::weak_ptr & file_segment_) + : key(key_), offset(offset_), file_segment(file_segment_) {} + CacheMetadata::Key key; size_t offset; /// We keep weak pointer to file segment diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 74801b68f73..13e46bc1023 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -29,6 +29,8 @@ public: struct PathWithInfo { + PathWithInfo() = default; + PathWithInfo(const String & path_, const std::optional & info_) : path(path_), info(info_) {} String path; std::optional info; }; From b8cac9499d01bd51e4b8a669c7d23104c656dc7c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 26 Jul 2023 13:18:09 +0000 Subject: [PATCH 29/72] Add tests to reproduce the problem --- .../0_stateless/02833_local_with_dialect.reference | 2 ++ tests/queries/0_stateless/02833_local_with_dialect.sh | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02833_local_with_dialect.reference create mode 100755 tests/queries/0_stateless/02833_local_with_dialect.sh diff --git a/tests/queries/0_stateless/02833_local_with_dialect.reference b/tests/queries/0_stateless/02833_local_with_dialect.reference new file mode 100644 index 00000000000..dbb67375997 --- /dev/null +++ b/tests/queries/0_stateless/02833_local_with_dialect.reference @@ -0,0 +1,2 @@ +0 +[?2004h[?2004lBye. diff --git a/tests/queries/0_stateless/02833_local_with_dialect.sh b/tests/queries/0_stateless/02833_local_with_dialect.sh new file mode 100755 index 00000000000..2a2e1b09459 --- /dev/null +++ b/tests/queries/0_stateless/02833_local_with_dialect.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +echo "exit" | ${CLICKHOUSE_LOCAL} --query "from s\"SELECT * FROM numbers(1)\"" --dialect prql --interactive From 3a6aaa29c9db0db1bc2875b7323c334148da387f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 26 Jul 2023 15:25:57 +0000 Subject: [PATCH 30/72] Do not load suggestions in case not ClickHouse dialects --- programs/local/LocalServer.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3c2a8ae3152..6ac7edaf1d9 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -569,7 +569,9 @@ void LocalServer::processConfig() } print_stack_trace = config().getBool("stacktrace", false); - load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false); + const std::string clickhouse_dialect{"clickhouse"}; + load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false) + && config().getString("dialect", clickhouse_dialect) == clickhouse_dialect; auto logging = (config().has("logger.console") || config().has("logger.level") From 5aeeaebd00c90385b1d9d8e3c6b13d298240c752 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 26 Jul 2023 17:09:23 -0300 Subject: [PATCH 31/72] init and destroy ares channel on demand.. --- src/Common/CaresPTRResolver.cpp | 71 ++++++++++--------- src/Common/CaresPTRResolver.h | 29 ++++---- .../tests/gtest_dns_reverse_resolve.cpp | 35 ++++----- 3 files changed, 71 insertions(+), 64 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index fee4d01eb37..bf61e940745 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -41,9 +41,25 @@ namespace DB } } - std::mutex CaresPTRResolver::mutex; + struct AresChannelRAII + { + AresChannelRAII() + { + if (ares_init(&channel) != ARES_SUCCESS) + { + throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to initialize c-ares channel"); + } + } - CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) : channel(nullptr) + ~AresChannelRAII() + { + ares_destroy(channel); + } + + ares_channel channel; + }; + + CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) { /* * ares_library_init is not thread safe. Currently, the only other usage of c-ares seems to be in grpc. @@ -57,34 +73,22 @@ namespace DB * */ static const auto library_init_result = ares_library_init(ARES_LIB_INIT_ALL); - if (library_init_result != ARES_SUCCESS || ares_init(&channel) != ARES_SUCCESS) + if (library_init_result != ARES_SUCCESS) { throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to initialize c-ares"); } } - CaresPTRResolver::~CaresPTRResolver() - { - ares_destroy(channel); - /* - * Library initialization is currently done only once in the constructor. Multiple instances of CaresPTRResolver - * will be used in the lifetime of ClickHouse, thus it's problematic to have de-init here. - * In a practical view, it makes little to no sense to de-init a DNS library since DNS requests will happen - * until the end of the program. Hence, ares_library_cleanup() will not be called. - * */ - } - std::unordered_set CaresPTRResolver::resolve(const std::string & ip) { - std::lock_guard guard(mutex); + AresChannelRAII channel_raii; std::unordered_set ptr_records; - resolve(ip, ptr_records); + resolve(ip, ptr_records, channel_raii.channel); - if (!wait_and_process()) + if (!wait_and_process(channel_raii.channel)) { - cancel_requests(); throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to complete reverse DNS query for IP {}", ip); } @@ -93,22 +97,21 @@ namespace DB std::unordered_set CaresPTRResolver::resolve_v6(const std::string & ip) { - std::lock_guard guard(mutex); + AresChannelRAII channel_raii; std::unordered_set ptr_records; - resolve_v6(ip, ptr_records); + resolve_v6(ip, ptr_records, channel_raii.channel); - if (!wait_and_process()) + if (!wait_and_process(channel_raii.channel)) { - cancel_requests(); throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to complete reverse DNS query for IP {}", ip); } return ptr_records; } - void CaresPTRResolver::resolve(const std::string & ip, std::unordered_set & response) + void CaresPTRResolver::resolve(const std::string & ip, std::unordered_set & response, ares_channel channel) { in_addr addr; @@ -117,7 +120,7 @@ namespace DB ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); } - void CaresPTRResolver::resolve_v6(const std::string & ip, std::unordered_set & response) + void CaresPTRResolver::resolve_v6(const std::string & ip, std::unordered_set & response, ares_channel channel) { in6_addr addr; inet_pton(AF_INET6, ip.c_str(), &addr); @@ -125,15 +128,15 @@ namespace DB ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET6, callback, &response); } - bool CaresPTRResolver::wait_and_process() + bool CaresPTRResolver::wait_and_process(ares_channel channel) { int sockets[ARES_GETSOCK_MAXNUM]; pollfd pollfd[ARES_GETSOCK_MAXNUM]; while (true) { - auto readable_sockets = get_readable_sockets(sockets, pollfd); - auto timeout = calculate_timeout(); + auto readable_sockets = get_readable_sockets(sockets, pollfd, channel); + auto timeout = calculate_timeout(channel); int number_of_fds_ready = 0; if (!readable_sockets.empty()) @@ -158,11 +161,11 @@ namespace DB if (number_of_fds_ready > 0) { - process_readable_sockets(readable_sockets); + process_readable_sockets(readable_sockets, channel); } else { - process_possible_timeout(); + process_possible_timeout(channel); break; } } @@ -170,12 +173,12 @@ namespace DB return true; } - void CaresPTRResolver::cancel_requests() + void CaresPTRResolver::cancel_requests(ares_channel channel) { ares_cancel(channel); } - std::span CaresPTRResolver::get_readable_sockets(int * sockets, pollfd * pollfd) + std::span CaresPTRResolver::get_readable_sockets(int * sockets, pollfd * pollfd, ares_channel channel) { int sockets_bitmask = ares_getsock(channel, sockets, ARES_GETSOCK_MAXNUM); @@ -205,7 +208,7 @@ namespace DB return std::span(pollfd, number_of_sockets_to_poll); } - int64_t CaresPTRResolver::calculate_timeout() + int64_t CaresPTRResolver::calculate_timeout(ares_channel channel) { timeval tv; if (auto * tvp = ares_timeout(channel, nullptr, &tv)) @@ -218,14 +221,14 @@ namespace DB return 0; } - void CaresPTRResolver::process_possible_timeout() + void CaresPTRResolver::process_possible_timeout(ares_channel channel) { /* Call ares_process() unconditonally here, even if we simply timed out above, as otherwise the ares name resolve won't timeout! */ ares_process_fd(channel, ARES_SOCKET_BAD, ARES_SOCKET_BAD); } - void CaresPTRResolver::process_readable_sockets(std::span readable_sockets) + void CaresPTRResolver::process_readable_sockets(std::span readable_sockets, ares_channel channel) { for (auto readable_socket : readable_sockets) { diff --git a/src/Common/CaresPTRResolver.h b/src/Common/CaresPTRResolver.h index 454509ae43c..24a5e422ca8 100644 --- a/src/Common/CaresPTRResolver.h +++ b/src/Common/CaresPTRResolver.h @@ -28,32 +28,35 @@ namespace DB public: explicit CaresPTRResolver(provider_token); - ~CaresPTRResolver() override; + + /* + * Library initialization is currently done only once in the constructor. Multiple instances of CaresPTRResolver + * will be used in the lifetime of ClickHouse, thus it's problematic to have de-init here. + * In a practical view, it makes little to no sense to de-init a DNS library since DNS requests will happen + * until the end of the program. Hence, ares_library_cleanup() will not be called. + * */ + ~CaresPTRResolver() override = default; std::unordered_set resolve(const std::string & ip) override; std::unordered_set resolve_v6(const std::string & ip) override; private: - bool wait_and_process(); + bool wait_and_process(ares_channel channel); - void cancel_requests(); + void cancel_requests(ares_channel channel); - void resolve(const std::string & ip, std::unordered_set & response); + void resolve(const std::string & ip, std::unordered_set & response, ares_channel channel); - void resolve_v6(const std::string & ip, std::unordered_set & response); + void resolve_v6(const std::string & ip, std::unordered_set & response, ares_channel channel); - std::span get_readable_sockets(int * sockets, pollfd * pollfd); + std::span get_readable_sockets(int * sockets, pollfd * pollfd, ares_channel channel); - int64_t calculate_timeout(); + int64_t calculate_timeout(ares_channel channel); - void process_possible_timeout(); + void process_possible_timeout(ares_channel channel); - void process_readable_sockets(std::span readable_sockets); - - ares_channel channel; - - static std::mutex mutex; + void process_readable_sockets(std::span readable_sockets, ares_channel channel); }; } diff --git a/src/Common/tests/gtest_dns_reverse_resolve.cpp b/src/Common/tests/gtest_dns_reverse_resolve.cpp index 08351564eaf..de33deddac3 100644 --- a/src/Common/tests/gtest_dns_reverse_resolve.cpp +++ b/src/Common/tests/gtest_dns_reverse_resolve.cpp @@ -9,34 +9,35 @@ namespace DB { TEST(Common, ReverseDNS) { - auto addresses = std::vector({ - "8.8.8.8", "2001:4860:4860::8888", // dns.google - "142.250.219.35", // google.com - "157.240.12.35", // facebook - "208.84.244.116", "2600:1419:c400::214:c410", //www.terra.com.br, - "127.0.0.1", "::1" - }); - auto func = [&]() { // Good random seed, good engine auto rnd1 = std::mt19937(std::random_device{}()); - for (int i = 0; i < 50; ++i) + for (int i = 0; i < 10; ++i) { auto & dns_resolver_instance = DNSResolver::instance(); -// unfortunately, DNS cache can't be disabled because we might end up causing a DDoS attack -// dns_resolver_instance.setDisableCacheFlag(); + dns_resolver_instance.setDisableCacheFlag(); - auto addr_index = rnd1() % addresses.size(); + auto val1 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val2 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val3 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val4 = rnd1() % static_cast((pow(2, 31) - 1)); - [[maybe_unused]] auto result = dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ addresses[addr_index] }); + uint32_t ipv4_buffer[1] = { + static_cast(val1) + }; -// will not assert either because some of the IP addresses might change in the future and -// this test will become flaky -// ASSERT_TRUE(!result.empty()); + uint32_t ipv6_buffer[4] = { + static_cast(val1), + static_cast(val2), + static_cast(val3), + static_cast(val4) + }; + + dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ ipv4_buffer, sizeof(ipv4_buffer)}); + dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ ipv6_buffer, sizeof(ipv6_buffer)}); } - }; auto number_of_threads = 200u; From d35c87c1bdf4cba8848e94acd0ed8ec1f6b34502 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 05:06:16 +0000 Subject: [PATCH 32/72] allow positional options for clickhouse-local and populate global udf settings --- programs/local/LocalServer.cpp | 15 +++++++++++++++ src/Client/ClientBase.cpp | 5 ----- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3c2a8ae3152..3ee268aed0e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -266,6 +266,10 @@ void LocalServer::tryInitPath() global_context->setUserFilesPath(""); // user's files are everywhere + std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/"); + global_context->setUserScriptsPath(user_scripts_path); + fs::create_directories(user_scripts_path); + /// top_level_domains_lists const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/"); if (!top_level_domains_path.empty()) @@ -490,6 +494,17 @@ try applyCmdSettings(global_context); + /// try to load user defined executable functions, throw on error and die + try + { + global_context->loadOrReloadUserDefinedExecutableFunctions(config()); + } + catch (...) + { + tryLogCurrentException(&logger(), "Caught exception while loading user defined executable functions."); + throw; + } + if (is_interactive) { clearTerminal(); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 3e964d5c6a3..496fc8fce0a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2624,11 +2624,6 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); } - /// Check positional options (options after ' -- ', ex: clickhouse-client -- ). - unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); - if (unrecognized_options.size() > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); - po::store(parsed, options); } From 65ffe91bf26a3429fe691c755736867e7819d2f5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 07:13:26 +0200 Subject: [PATCH 33/72] Fix double whitespace --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index b2c75db0e54..c661e6b782d 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1011,7 +1011,7 @@ private: using ConfluentSchemaRegistry = AvroConfluentRowInputFormat::SchemaRegistry; #define SCHEMA_REGISTRY_CACHE_MAX_SIZE 1000 /// Cache of Schema Registry URL -> SchemaRegistry -static CacheBase schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE); +static CacheBase schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE); static std::shared_ptr getConfluentSchemaRegistry(const FormatSettings & format_settings) { From d2d7139da3af470a49267047f0ea45f652d59e45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 08:58:23 +0200 Subject: [PATCH 34/72] Changelog for 23.7 --- CHANGELOG.md | 175 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 175 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bf6b309ef2c..878edfa4add 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v23.7, 2023-07-27](#237)**
**[ClickHouse release v23.6, 2023-06-30](#236)**
**[ClickHouse release v23.5, 2023-06-08](#235)**
**[ClickHouse release v23.4, 2023-04-26](#234)**
@@ -9,6 +10,180 @@ # 2023 Changelog +### ClickHouse release 23.7, 2023-07-27 + +#### Backward Incompatible Change +* Add `NAMED COLLECTION` access type (aliases `USE NAMED COLLECTION`, `NAMED COLLECTION USAGE`). This PR is backward incompatible because this access type is disabled by default (because a parent access type `NAMED COLLECTION ADMIN` is disabled by default as well). Proposed in [#50277](https://github.com/ClickHouse/ClickHouse/issues/50277). To grant use `GRANT NAMED COLLECTION ON collection_name TO user` or `GRANT NAMED COLLECTION ON * TO user`, to be able to give these grants `named_collection_admin` is required in config (previously it was named `named_collection_control`, so will remain as an alias). [#50625](https://github.com/ClickHouse/ClickHouse/pull/50625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixing a typo in the `system.parts` column name `last_removal_attemp_time`. Now it is named `last_removal_attempt_time`. [#52104](https://github.com/ClickHouse/ClickHouse/pull/52104) ([filimonov](https://github.com/filimonov)). +* Bump version of the distributed_ddl_entry_format_version to 5 by default (enables opentelemetry and initial_query_idd pass through). This will not allow to process existing entries for distributed DDL after *downgrade* (but note, that usually there should be no such unprocessed entries). [#52128](https://github.com/ClickHouse/ClickHouse/pull/52128) ([Azat Khuzhin](https://github.com/azat)). +* Check projection metadata the same way we check ordinary metadata. This change may prevent the server from starting in case there was a table with an invalid projection. An example is a projection that created positional columns in PK (e.g. `projection p (select * order by 1, 4)` which is not allowed in table PK and can cause a crash during insert/merge). Drop such projections before the update. Fixes [#52353](https://github.com/ClickHouse/ClickHouse/issues/52353). [#52361](https://github.com/ClickHouse/ClickHouse/pull/52361) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Added support for PRQL as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). +* Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). +* Kafka connector can fetch Avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). +* Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Add a column `is_obsolete` to `system.settings` and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). +* Implement support of encrypted elements in configuration file. Added possibility to use encrypted text in leaf elements of configuration file. The text is encrypted using encryption codecs from `` section. [#50986](https://github.com/ClickHouse/ClickHouse/pull/50986) ([Roman Vasin](https://github.com/rvasin)). +* Grace Hash Join algorithm is now applicable to FULL and RIGHT JOINs. [#49483](https://github.com/ClickHouse/ClickHouse/issues/49483). [#51013](https://github.com/ClickHouse/ClickHouse/pull/51013) ([lgbo](https://github.com/lgbo-ustc)). +* Add `SYSTEM STOP LISTEN` query for more graceful termination. Closes [#47972](https://github.com/ClickHouse/ClickHouse/issues/47972). [#51016](https://github.com/ClickHouse/ClickHouse/pull/51016) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add `input_format_csv_allow_variable_number_of_columns` options. [#51273](https://github.com/ClickHouse/ClickHouse/pull/51273) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Another boring feature: add function `substring_index`, as in Spark or MySQL. [#51472](https://github.com/ClickHouse/ClickHouse/pull/51472) ([李扬](https://github.com/taiyang-li)). +* A system table `jemalloc_bins` to show stats for jemalloc bins. Example `SELECT *, size * (nmalloc - ndalloc) AS allocated_bytes FROM system.jemalloc_bins WHERE allocated_bytes > 0 ORDER BY allocated_bytes DESC LIMIT 10`. Enjoy. [#51674](https://github.com/ClickHouse/ClickHouse/pull/51674) ([Alexander Gololobov](https://github.com/davenger)). +* Add `RowBinaryWithDefaults` format with extra byte before each column as a flag for using the column's default value. Closes [#50854](https://github.com/ClickHouse/ClickHouse/issues/50854). [#51695](https://github.com/ClickHouse/ClickHouse/pull/51695) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `default_temporary_table_engine` setting. Same as `default_table_engine` but for temporary tables. [#51292](https://github.com/ClickHouse/ClickHouse/issues/51292). [#51708](https://github.com/ClickHouse/ClickHouse/pull/51708) ([velavokr](https://github.com/velavokr)). +* Added new `initcap` / `initcapUTF8` functions which convert the first letter of each word to upper case and the rest to lower case. [#51735](https://github.com/ClickHouse/ClickHouse/pull/51735) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create table now supports `PRIMARY KEY` syntax in column definition. Columns are added to primary index in the same order columns are defined. [#51881](https://github.com/ClickHouse/ClickHouse/pull/51881) ([Ilya Yatsishin](https://github.com/qoega)). +* Added the possibility to use date and time format specifiers in log and error log file names, either in config files (`log` and `errorlog` tags) or command line arguments (`--log-file` and `--errorlog-file`). [#51945](https://github.com/ClickHouse/ClickHouse/pull/51945) ([Victor Krasnov](https://github.com/sirvickr)). +* Added Peak Memory Usage statistic to HTTP headers. [#51946](https://github.com/ClickHouse/ClickHouse/pull/51946) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added new `hasSubsequence` (+`CaseInsensitive` and `UTF8` versions) functions to match subsequences in strings. [#52050](https://github.com/ClickHouse/ClickHouse/pull/52050) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Add `array_agg` as alias of `groupArray` for PostgreSQL compatibility. Closes [#52100](https://github.com/ClickHouse/ClickHouse/issues/52100). ### Documentation entry for user-facing changes. [#52135](https://github.com/ClickHouse/ClickHouse/pull/52135) ([flynn](https://github.com/ucasfl)). +* Add `any_value` as a compatibility alias for `any` aggregate function. Closes [#52140](https://github.com/ClickHouse/ClickHouse/issues/52140). [#52147](https://github.com/ClickHouse/ClickHouse/pull/52147) ([flynn](https://github.com/ucasfl)). +* Add aggregate function `array_concat_agg` for compatibility with BigQuery, it's alias of `groupArrayArray`. Closes [#52139](https://github.com/ClickHouse/ClickHouse/issues/52139). [#52149](https://github.com/ClickHouse/ClickHouse/pull/52149) ([flynn](https://github.com/ucasfl)). +* Add `OCTET_LENGTH` as an alias to `length`. Closes [#52153](https://github.com/ClickHouse/ClickHouse/issues/52153). [#52176](https://github.com/ClickHouse/ClickHouse/pull/52176) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Added `firstLine` function to extract the first line from the multi-line string. This closes [#51172](https://github.com/ClickHouse/ClickHouse/issues/51172). [#52209](https://github.com/ClickHouse/ClickHouse/pull/52209) ([Mikhail Koviazin](https://github.com/mkmkme)). +* Implement KQL-style formatting for the `Interval` data type. This is only needed for compatibility with the `Kusto` query language. [#45671](https://github.com/ClickHouse/ClickHouse/pull/45671) ([ltrk2](https://github.com/ltrk2)). +* Added query `SYSTEM FLUSH ASYNC INSERT QUEUE` which flushes all pending asynchronous inserts to the destination tables. Added a server-side setting `async_insert_queue_flush_on_shutdown` (`true` by default) which determines whether to flush queue of asynchronous inserts on graceful shutdown. Setting `async_insert_threads` is now a server-side setting. [#49160](https://github.com/ClickHouse/ClickHouse/pull/49160) ([Anton Popov](https://github.com/CurtizJ)). +* Aliases `current_database` and a new function `current_schemas` for compatibility with PostgreSQL. [#51076](https://github.com/ClickHouse/ClickHouse/pull/51076) ([Pedro Riera](https://github.com/priera)). +* Add alias for functions `today` (now available under the `curdate`/`current_date` names) and `now` (`current_timestamp`). [#52106](https://github.com/ClickHouse/ClickHouse/pull/52106) ([Lloyd-Pottiger](https://github.com/Lloyd-Pottiger)). +* Support `async_deduplication_token` for async insert. [#52136](https://github.com/ClickHouse/ClickHouse/pull/52136) ([Han Fei](https://github.com/hanfei1991)). +* Add new setting `disable_url_encoding` that allows to disable decoding/encoding path in uri in URL engine. [#52337](https://github.com/ClickHouse/ClickHouse/pull/52337) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Performance Improvement +* Writing parquet files is 10x faster, it's multi-threaded now. Almost the same speed as reading. [#49367](https://github.com/ClickHouse/ClickHouse/pull/49367) ([Michael Kolupaev](https://github.com/al13n321)). +* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable `move_all_conditions_to_prewhere` and `enable_multiple_prewhere_read_steps` settings by default. [#46365](https://github.com/ClickHouse/ClickHouse/pull/46365) ([Alexander Gololobov](https://github.com/davenger)). +* Improves performance of some queries by tuning allocator. [#46416](https://github.com/ClickHouse/ClickHouse/pull/46416) ([Azat Khuzhin](https://github.com/azat)). +* Now we use fixed-size tasks in `MergeTreePrefetchedReadPool` as in `MergeTreeReadPool`. Also from now we use connection pool for S3 requests. [#49732](https://github.com/ClickHouse/ClickHouse/pull/49732) ([Nikita Taranov](https://github.com/nickitat)). +* More pushdown to the right side of join. [#50532](https://github.com/ClickHouse/ClickHouse/pull/50532) ([Nikita Taranov](https://github.com/nickitat)). +* Improve grace_hash join by reserving hash table's size (resubmit). [#50875](https://github.com/ClickHouse/ClickHouse/pull/50875) ([lgbo](https://github.com/lgbo-ustc)). +* Waiting on lock in `OpenedFileCache` could be noticeable sometimes. We sharded it into multiple sub-maps (each with its own lock) to avoid contention. [#51341](https://github.com/ClickHouse/ClickHouse/pull/51341) ([Nikita Taranov](https://github.com/nickitat)). +* Move conditions with primary key columns to the end of PREWHERE chain. The idea is that conditions with PK columns are likely to be used in PK analysis and will not contribute much more to PREWHERE filtering. [#51958](https://github.com/ClickHouse/ClickHouse/pull/51958) ([Alexander Gololobov](https://github.com/davenger)). +* Speed up `COUNT(DISTINCT)` for String types by inlining SipHash. The performance experiments of *OnTime* on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) show that this change could bring an improvement of *11.6%* to the QPS of the query *Q8* while having no impact on others. [#52036](https://github.com/ClickHouse/ClickHouse/pull/52036) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Enable `allow_vertical_merges_from_compact_to_wide_parts` by default. It will save memory usage during merges. [#52295](https://github.com/ClickHouse/ClickHouse/pull/52295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis which invalidates primary keys. This issue only exists when `query_plan_optimize_primary_key = 1, query_plan_optimize_projection = 1`. This fixes [#48823](https://github.com/ClickHouse/ClickHouse/issues/48823). This fixes [#51173](https://github.com/ClickHouse/ClickHouse/issues/51173). [#52308](https://github.com/ClickHouse/ClickHouse/pull/52308) ([Amos Bird](https://github.com/amosbird)). +* Reduce the number of syscalls in `FileCache::loadMetadata` - this speeds up server startup if the filesystem cache is configured. [#52435](https://github.com/ClickHouse/ClickHouse/pull/52435) ([Raúl Marín](https://github.com/Algunenano)). +* Allow to have strict lower boundary for file segment size by downloading remaining data in the background. Minimum size of file segment (if actual file size is bigger) is configured as cache configuration setting `boundary_alignment`, by default `4Mi`. Number of background threads are configured as cache configuration setting `background_download_threads`, by default `2`. Also `max_file_segment_size` was increased from `8Mi` to `32Mi` in this PR. [#51000](https://github.com/ClickHouse/ClickHouse/pull/51000) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Decreased default timeouts for S3 from 30 seconds to 3 seconds, and for other HTTP from 180 seconds to 30 seconds. [#51171](https://github.com/ClickHouse/ClickHouse/pull/51171) ([Michael Kolupaev](https://github.com/al13n321)). +* New setting `merge_tree_determine_task_size_by_prewhere_columns` added. If set to `true` only sizes of the columns from `PREWHERE` section will be considered to determine reading task size. Otherwise all the columns from query are considered. [#52606](https://github.com/ClickHouse/ClickHouse/pull/52606) ([Nikita Taranov](https://github.com/nickitat)). + +#### Improvement +* Use read_bytes/total_bytes_to_read for progress bar in s3/file/url/... table functions for better progress indication. [#51286](https://github.com/ClickHouse/ClickHouse/pull/51286) ([Kruglov Pavel](https://github.com/Avogar)). +* Introduce a table setting `wait_for_unique_parts_send_before_shutdown_ms` which specify the amount of time replica will wait before closing interserver handler for replicated sends. Also fix inconsistency with shutdown of tables and interserver handlers: now server shutdown tables first and only after it shut down interserver handlers. [#51851](https://github.com/ClickHouse/ClickHouse/pull/51851) ([alesapin](https://github.com/alesapin)). +* Allow SQL standard `FETCH` without `OFFSET`. See https://antonz.org/sql-fetch/. [#51293](https://github.com/ClickHouse/ClickHouse/pull/51293) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow filtering HTTP headers for the URL/S3 table functions with the new `http_forbid_headers` section in config. Both exact matching and regexp filters are available. [#51038](https://github.com/ClickHouse/ClickHouse/pull/51038) ([Nikolay Degterinsky](https://github.com/evillique)). +* Don't show messages about `16 EiB` free space in logs, as they don't make sense. This closes [#49320](https://github.com/ClickHouse/ClickHouse/issues/49320). [#49342](https://github.com/ClickHouse/ClickHouse/pull/49342) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Properly check the limit for the `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block`. This is needed for generic query fuzzer. [#49343](https://github.com/ClickHouse/ClickHouse/pull/49343) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix two issues in `geoHash` functions. [#50066](https://github.com/ClickHouse/ClickHouse/pull/50066) ([李扬](https://github.com/taiyang-li)). +* Log async insert flush queries into `system.query_log`. [#51160](https://github.com/ClickHouse/ClickHouse/pull/51160) ([Raúl Marín](https://github.com/Algunenano)). +* Functions `date_diff` and `age` now support millisecond/microsecond unit and work with microsecond precision. [#51291](https://github.com/ClickHouse/ClickHouse/pull/51291) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Improve parsing of path in clickhouse-keeper-client. [#51359](https://github.com/ClickHouse/ClickHouse/pull/51359) ([Azat Khuzhin](https://github.com/azat)). +* A third-party product depending on ClickHouse (Gluten: a Plugin to Double SparkSQL's Performance) had a bug. This fix avoids heap overflow in that third-party product while reading from HDFS. [#51386](https://github.com/ClickHouse/ClickHouse/pull/51386) ([李扬](https://github.com/taiyang-li)). +* Add ability to disable native copy for S3 (setting for BACKUP/RESTORE `allow_s3_native_copy`, and `s3_allow_native_copy` for `s3`/`s3_plain` disks). [#51448](https://github.com/ClickHouse/ClickHouse/pull/51448) ([Azat Khuzhin](https://github.com/azat)). +* Add column `primary_key_size` to `system.parts` table to show compressed primary key size on disk. Closes [#51400](https://github.com/ClickHouse/ClickHouse/issues/51400). [#51496](https://github.com/ClickHouse/ClickHouse/pull/51496) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Allow running `clickhouse-local` without procfs, without home directory existing, and without name resolution plugins from glibc. [#51518](https://github.com/ClickHouse/ClickHouse/pull/51518) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add placeholder `%a` for rull filename in rename_files_after_processing setting. [#51603](https://github.com/ClickHouse/ClickHouse/pull/51603) ([Kruglov Pavel](https://github.com/Avogar)). +* Add column `modification_time` into `system.parts_columns`. [#51685](https://github.com/ClickHouse/ClickHouse/pull/51685) ([Azat Khuzhin](https://github.com/azat)). +* Add new setting `input_format_csv_use_default_on_bad_values` to CSV format that allows to insert default value when parsing of a single field failed. [#51716](https://github.com/ClickHouse/ClickHouse/pull/51716) ([KevinyhZou](https://github.com/KevinyhZou)). +* Added a crash log flush to the disk after the unexpected crash. [#51720](https://github.com/ClickHouse/ClickHouse/pull/51720) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix behavior in dashboard page where errors unrelated to authentication are not shown. Also fix 'overlapping' chart behavior. [#51744](https://github.com/ClickHouse/ClickHouse/pull/51744) ([Zach Naimon](https://github.com/ArctypeZach)). +* Allow UUID to UInt128 conversion. [#51765](https://github.com/ClickHouse/ClickHouse/pull/51765) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added support for function `range` of Nullable arguments. [#51767](https://github.com/ClickHouse/ClickHouse/pull/51767) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Convert condition like `toyear(x) = c` to `c1 <= x < c2`. [#51795](https://github.com/ClickHouse/ClickHouse/pull/51795) ([Han Fei](https://github.com/hanfei1991)). +* Improve MySQL compatibility of the statement `SHOW INDEX`. [#51796](https://github.com/ClickHouse/ClickHouse/pull/51796) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix `use_structure_from_insertion_table_in_table_functions` does not work with `MATERIALIZED` and `ALIAS` columns. Closes [#51817](https://github.com/ClickHouse/ClickHouse/issues/51817). Closes [#51019](https://github.com/ClickHouse/ClickHouse/issues/51019). [#51825](https://github.com/ClickHouse/ClickHouse/pull/51825) ([flynn](https://github.com/ucasfl)). +* Cache dictionary now requests only unique keys from source. Closes [#51762](https://github.com/ClickHouse/ClickHouse/issues/51762). [#51853](https://github.com/ClickHouse/ClickHouse/pull/51853) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed the case when settings were not applied for EXPLAIN query when FORMAT was provided. [#51859](https://github.com/ClickHouse/ClickHouse/pull/51859) ([Nikita Taranov](https://github.com/nickitat)). +* Allow SETTINGS before FORMAT in DESCRIBE TABLE query for compatibility with SELECT query. Closes [#51544](https://github.com/ClickHouse/ClickHouse/issues/51544). [#51899](https://github.com/ClickHouse/ClickHouse/pull/51899) ([Nikolay Degterinsky](https://github.com/evillique)). +* Var-Int encoded integers (e.g. used by the native protocol) can now use the full 64-bit range. 3rd party clients are advised to update their var-int code accordingly. [#51905](https://github.com/ClickHouse/ClickHouse/pull/51905) ([Robert Schulze](https://github.com/rschu1ze)). +* Update certificates when they change without the need to manually SYSTEM RELOAD CONFIG. [#52030](https://github.com/ClickHouse/ClickHouse/pull/52030) ([Mike Kot](https://github.com/myrrc)). +* Added `allow_create_index_without_type` setting that allow to ignore `ADD INDEX` queries without specified `TYPE`. Standard SQL queries will just succeed without changing table schema. [#52056](https://github.com/ClickHouse/ClickHouse/pull/52056) ([Ilya Yatsishin](https://github.com/qoega)). +* Log messages are written to the `system.text_log` from the server startup. [#52113](https://github.com/ClickHouse/ClickHouse/pull/52113) ([Dmitry Kardymon](https://github.com/kardymonds)). +* In cases where the HTTP endpoint has multiple IP addresses and the first of them is unreachable, a timeout exception was thrown. Made session creation with handling all resolved endpoints. [#52116](https://github.com/ClickHouse/ClickHouse/pull/52116) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Avro input format now supports Union even if it contains only a single type. Closes [#52131](https://github.com/ClickHouse/ClickHouse/issues/52131). [#52137](https://github.com/ClickHouse/ClickHouse/pull/52137) ([flynn](https://github.com/ucasfl)). +* Add setting `optimize_use_implicit_projections` to disable implicit projections (currently only `min_max_count` projection). [#52152](https://github.com/ClickHouse/ClickHouse/pull/52152) ([Amos Bird](https://github.com/amosbird)). +* It was possible to use the function `hasToken` for infinite loop. Now this possibility is removed. This closes [#52156](https://github.com/ClickHouse/ClickHouse/issues/52156). [#52160](https://github.com/ClickHouse/ClickHouse/pull/52160) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Create ZK ancestors optimistically. [#52195](https://github.com/ClickHouse/ClickHouse/pull/52195) ([Raúl Marín](https://github.com/Algunenano)). +* Fix [#50582](https://github.com/ClickHouse/ClickHouse/issues/50582). Avoid the `Not found column ... in block` error in some cases of reading in-order and constants. [#52259](https://github.com/ClickHouse/ClickHouse/pull/52259) ([Chen768959](https://github.com/Chen768959)). +* Check whether S2 geo primitives are invalid as early as possible on ClickHouse side. This closes: [#27090](https://github.com/ClickHouse/ClickHouse/issues/27090). [#52260](https://github.com/ClickHouse/ClickHouse/pull/52260) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add back missing projection QueryAccessInfo when `query_plan_optimize_projection = 1`. This fixes [#50183](https://github.com/ClickHouse/ClickHouse/issues/50183) . This fixes [#50093](https://github.com/ClickHouse/ClickHouse/issues/50093). [#52327](https://github.com/ClickHouse/ClickHouse/pull/52327) ([Amos Bird](https://github.com/amosbird)). +* When `ZooKeeperRetriesControl` rethrows an error, it's more useful to see its original stack trace, not the one from `ZooKeeperRetriesControl` itself. [#52347](https://github.com/ClickHouse/ClickHouse/pull/52347) ([Vitaly Baranov](https://github.com/vitlibar)). +* Wait for zero copy replication lock even if some disks don't support it. [#52376](https://github.com/ClickHouse/ClickHouse/pull/52376) ([Raúl Marín](https://github.com/Algunenano)). +* Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). + +#### Experimental Feature +* Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). This syntax can be changed in this release. +* (experimental MaterializedMySQL) Fixed crash when `mysqlxx::Pool::Entry` is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) `CREATE TABLE ... AS SELECT` .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Introduced automatic conversion of text types to utf8 for MaterializedMySQL. [#52084](https://github.com/ClickHouse/ClickHouse/pull/52084) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Now unquoted UTF-8 strings are supported in DDL for MaterializedMySQL. [#52318](https://github.com/ClickHouse/ClickHouse/pull/52318) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). +* Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). +* The `session_timezone` setting (new in version 23.6) is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add integration test check with the enabled Analyzer. [#50926](https://github.com/ClickHouse/ClickHouse/pull/50926) [#52210](https://github.com/ClickHouse/ClickHouse/pull/52210) ([Dmitry Novik](https://github.com/novikd)). +* Reproducible builds for Rust. [#52395](https://github.com/ClickHouse/ClickHouse/pull/52395) ([Azat Khuzhin](https://github.com/azat)). +* Update Cargo dependencies. [#51721](https://github.com/ClickHouse/ClickHouse/pull/51721) ([Raúl Marín](https://github.com/Algunenano)). +* Make the function `CHColumnToArrowColumn::fillArrowArrayWithArrayColumnData` to work with nullable arrays, which are not possible in ClickHouse, but needed for Gluten. [#52112](https://github.com/ClickHouse/ClickHouse/pull/52112) ([李扬](https://github.com/taiyang-li)). +* We've updated the CCTZ library to master, but there are no user-visible changes. [#52124](https://github.com/ClickHouse/ClickHouse/pull/52124) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `system.licenses` table now includes the hard-forked library Poco. This closes [#52066](https://github.com/ClickHouse/ClickHouse/issues/52066). [#52127](https://github.com/ClickHouse/ClickHouse/pull/52127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that there are no cases of bad punctuation: whitespace before a comma like `Hello ,world` instead of `Hello, world`. [#52549](https://github.com/ClickHouse/ClickHouse/pull/52549) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix MaterializedPostgreSQL syncTables [#49698](https://github.com/ClickHouse/ClickHouse/pull/49698) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix projection with optimize_aggregators_of_group_by_keys [#49709](https://github.com/ClickHouse/ClickHouse/pull/49709) ([Amos Bird](https://github.com/amosbird)). +* Fix optimize_skip_unused_shards with JOINs [#51037](https://github.com/ClickHouse/ClickHouse/pull/51037) ([Azat Khuzhin](https://github.com/azat)). +* Fix formatDateTime() with fractional negative datetime64 [#51290](https://github.com/ClickHouse/ClickHouse/pull/51290) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Functions `hasToken*` were totally wrong. Add a test for [#43358](https://github.com/ClickHouse/ClickHouse/issues/43358) [#51378](https://github.com/ClickHouse/ClickHouse/pull/51378) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix optimization to move functions before sorting. [#51481](https://github.com/ClickHouse/ClickHouse/pull/51481) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix Block structure mismatch in Pipe::unitePipes for FINAL [#51492](https://github.com/ClickHouse/ClickHouse/pull/51492) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SIGSEGV for clusters with zero weight across all shards (fixes INSERT INTO FUNCTION clusterAllReplicas()) [#51545](https://github.com/ClickHouse/ClickHouse/pull/51545) ([Azat Khuzhin](https://github.com/azat)). +* Fix timeout for hedged requests [#51582](https://github.com/ClickHouse/ClickHouse/pull/51582) ([Azat Khuzhin](https://github.com/azat)). +* Fix logical error in ANTI join with NULL [#51601](https://github.com/ClickHouse/ClickHouse/pull/51601) ([vdimir](https://github.com/vdimir)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Do not apply PredicateExpressionsOptimizer for ASOF/ANTI join [#51633](https://github.com/ClickHouse/ClickHouse/pull/51633) ([vdimir](https://github.com/vdimir)). +* Fix async insert with deduplication for ReplicatedMergeTree using merging algorithms [#51676](https://github.com/ClickHouse/ClickHouse/pull/51676) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Fix segfault when create invalid EmbeddedRocksdb table [#51847](https://github.com/ClickHouse/ClickHouse/pull/51847) ([Duc Canh Le](https://github.com/canhld94)). +* Fix inserts into MongoDB tables [#51876](https://github.com/ClickHouse/ClickHouse/pull/51876) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix deadlock on DatabaseCatalog shutdown [#51908](https://github.com/ClickHouse/ClickHouse/pull/51908) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix error in subquery operators [#51922](https://github.com/ClickHouse/ClickHouse/pull/51922) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix async connect to hosts with multiple ips [#51934](https://github.com/ClickHouse/ClickHouse/pull/51934) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not remove inputs after ActionsDAG::merge [#51947](https://github.com/ClickHouse/ClickHouse/pull/51947) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` [#51954](https://github.com/ClickHouse/ClickHouse/pull/51954) ([vdimir](https://github.com/vdimir)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix for toDateTime64() for dates after 2283-12-31 [#52130](https://github.com/ClickHouse/ClickHouse/pull/52130) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix ORDER BY tuple of WINDOW functions [#52145](https://github.com/ClickHouse/ClickHouse/pull/52145) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis when aggregation expression contains monotonic functions [#52151](https://github.com/ClickHouse/ClickHouse/pull/52151) ([Amos Bird](https://github.com/amosbird)). +* Fix error in `groupArrayMoving` functions [#52161](https://github.com/ClickHouse/ClickHouse/pull/52161) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable direct join for range dictionary [#52187](https://github.com/ClickHouse/ClickHouse/pull/52187) ([Duc Canh Le](https://github.com/canhld94)). +* Fix sticky mutations test (and extremely rare race condition) [#52197](https://github.com/ClickHouse/ClickHouse/pull/52197) ([alesapin](https://github.com/alesapin)). +* Fix race in Web disk [#52211](https://github.com/ClickHouse/ClickHouse/pull/52211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in Connection::setAsyncCallback on unknown packet from server [#52219](https://github.com/ClickHouse/ClickHouse/pull/52219) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix temp data deletion on startup, add test [#52275](https://github.com/ClickHouse/ClickHouse/pull/52275) ([vdimir](https://github.com/vdimir)). +* Don't use minmax_count projections when counting nullable columns [#52297](https://github.com/ClickHouse/ClickHouse/pull/52297) ([Amos Bird](https://github.com/amosbird)). +* MergeTree/ReplicatedMergeTree should use server timezone for log entries [#52325](https://github.com/ClickHouse/ClickHouse/pull/52325) ([Azat Khuzhin](https://github.com/azat)). +* Fix parameterized view with cte and multiple usage [#52328](https://github.com/ClickHouse/ClickHouse/pull/52328) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Disable expression templates for time intervals [#52335](https://github.com/ClickHouse/ClickHouse/pull/52335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `apply_snapshot` in Keeper [#52358](https://github.com/ClickHouse/ClickHouse/pull/52358) ([Antonio Andelic](https://github.com/antonio2368)). +* Update build-osx.md [#52377](https://github.com/ClickHouse/ClickHouse/pull/52377) ([AlexBykovski](https://github.com/AlexBykovski)). +* Fix `countSubstrings()` hang with empty needle and a column haystack [#52409](https://github.com/ClickHouse/ClickHouse/pull/52409) ([Sergei Trifonov](https://github.com/serxa)). +* Fix normal projection with merge table [#52432](https://github.com/ClickHouse/ClickHouse/pull/52432) ([Amos Bird](https://github.com/amosbird)). +* Fix possible double-free in Aggregator [#52439](https://github.com/ClickHouse/ClickHouse/pull/52439) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed inserting into Buffer engine [#52440](https://github.com/ClickHouse/ClickHouse/pull/52440) ([Vasily Nemkov](https://github.com/Enmk)). +* The implementation of AnyHash was non-conformant. [#52448](https://github.com/ClickHouse/ClickHouse/pull/52448) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check recursion depth in OptimizedRegularExpression [#52451](https://github.com/ClickHouse/ClickHouse/pull/52451) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() [#52490](https://github.com/ClickHouse/ClickHouse/pull/52490) ([Azat Khuzhin](https://github.com/azat)). +* Fix abort in function `transform` [#52513](https://github.com/ClickHouse/ClickHouse/pull/52513) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix lightweight delete after drop of projection [#52517](https://github.com/ClickHouse/ClickHouse/pull/52517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error "Cannot drain connections: cancel first" [#52585](https://github.com/ClickHouse/ClickHouse/pull/52585) ([Kruglov Pavel](https://github.com/Avogar)). + + ### ClickHouse release 23.6, 2023-06-29 #### Backward Incompatible Change From ac51ade45c5581d031c382277b550b2fb2f873fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 27 Jul 2023 08:45:53 +0000 Subject: [PATCH 35/72] Do not run the test without Rust libraries --- tests/queries/0_stateless/02833_local_with_dialect.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02833_local_with_dialect.sh b/tests/queries/0_stateless/02833_local_with_dialect.sh index 2a2e1b09459..012a6d91269 100755 --- a/tests/queries/0_stateless/02833_local_with_dialect.sh +++ b/tests/queries/0_stateless/02833_local_with_dialect.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f61f36800ccd028bb9cf4ef402275006faf2facb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 27 Jul 2023 08:48:23 +0000 Subject: [PATCH 36/72] Fix style --- .../Formats/Impl/Parquet/PrepareForWrite.cpp | 14 +++++++------- src/Processors/Formats/Impl/Parquet/Write.cpp | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp index 0700fc8491c..bc4c9ca3b72 100644 --- a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -303,14 +303,14 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin } else { - types(T::INT32, C::UINT_8 , int_type(8 , false)); + types(T::INT32, C::UINT_8, int_type(8, false)); } break; case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break; case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break; case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break; - case TypeIndex::Int8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; - case TypeIndex::Int16: types(T::INT32, C::INT_16 , int_type(16, true)); break; + case TypeIndex::Int8: types(T::INT32, C::INT_8, int_type(8, true)); break; + case TypeIndex::Int16: types(T::INT32, C::INT_16, int_type(16, true)); break; case TypeIndex::Int32: types(T::INT32); break; case TypeIndex::Int64: types(T::INT64); break; case TypeIndex::Float32: types(T::FLOAT); break; @@ -319,8 +319,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin /// These don't have suitable parquet logical types, so we write them as plain numbers. /// (Parquet has "enums" but they're just strings, with nowhere to declare all possible enum /// values in advance as part of the data type.) - case TypeIndex::Enum8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; // Int8 - case TypeIndex::Enum16: types(T::INT32, C::INT_16 , int_type(16, true)); break; // Int16 + case TypeIndex::Enum8: types(T::INT32, C::INT_8, int_type(8, true)); break; // Int8 + case TypeIndex::Enum16: types(T::INT32, C::INT_16, int_type(16, true)); break; // Int16 case TypeIndex::IPv4: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 case TypeIndex::Date: types(T::INT32, C::UINT_16, int_type(16, false)); break; // UInt16 case TypeIndex::DateTime: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 @@ -392,8 +392,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin case TypeIndex::Int256: fixed_string(32); break; case TypeIndex::IPv6: fixed_string(16); break; - case TypeIndex::Decimal32: decimal(4 , getDecimalPrecision(*type), getDecimalScale(*type)); break; - case TypeIndex::Decimal64: decimal(8 , getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal32: decimal(4, getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal64: decimal(8, getDecimalPrecision(*type), getDecimalScale(*type)); break; case TypeIndex::Decimal128: decimal(16, getDecimalPrecision(*type), getDecimalScale(*type)); break; case TypeIndex::Decimal256: decimal(32, getDecimalPrecision(*type), getDecimalScale(*type)); break; diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 47ef0c53ab5..5ebf2be76d2 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -755,20 +755,20 @@ void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & option writeColumnImpl(s, options, out, ConverterNumeric, bool, bool>(s.primitive_column)); else - N(UInt8 , Int32Type); + N(UInt8, Int32Type); break; case TypeIndex::UInt16 : N(UInt16, Int32Type); break; case TypeIndex::UInt32 : N(UInt32, Int32Type); break; case TypeIndex::UInt64 : N(UInt64, Int64Type); break; - case TypeIndex::Int8 : N(Int8 , Int32Type); break; - case TypeIndex::Int16 : N(Int16 , Int32Type); break; - case TypeIndex::Int32 : N(Int32 , Int32Type); break; - case TypeIndex::Int64 : N(Int64 , Int64Type); break; + case TypeIndex::Int8 : N(Int8, Int32Type); break; + case TypeIndex::Int16 : N(Int16, Int32Type); break; + case TypeIndex::Int32 : N(Int32, Int32Type); break; + case TypeIndex::Int64 : N(Int64, Int64Type); break; - case TypeIndex::Enum8: N(Int8 , Int32Type); break; - case TypeIndex::Enum16: N(Int16 , Int32Type); break; + case TypeIndex::Enum8: N(Int8, Int32Type); break; + case TypeIndex::Enum16: N(Int16, Int32Type); break; case TypeIndex::Date: N(UInt16, Int32Type); break; - case TypeIndex::Date32: N(Int32 , Int32Type); break; + case TypeIndex::Date32: N(Int32, Int32Type); break; case TypeIndex::DateTime: N(UInt32, Int32Type); break; #undef N From 2b18872e86898fe0c0ee40ddecf05c29088a7aca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 08:55:19 +0000 Subject: [PATCH 37/72] Incorporate review feedback --- docs/en/operations/system-tables/query_log.md | 8 ++++---- src/Interpreters/Cache/QueryCache.h | 8 ++++---- src/Interpreters/QueryLog.cpp | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index b8dc0c0224c..835c79129de 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -112,10 +112,10 @@ Columns: - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - - `'None' = 1` = The query result was neither written into nor read from the query cache. - - `'Write' = 1` = The query result was written into the query cache. - - `'Read' = 1` = The query result was read from the query cache. - - `'Unknown' = 1` = Unknown status. + - `'Unknown' = 1` = Status unknown. + - `'None' = 2` = The query result was neither written into nor read from the query cache. + - `'Write' = 3` = The query result was written into the query cache. + - `'Read' = 4` = The query result was read from the query cache. **Example** diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 973015b8003..5fe756268f2 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -27,10 +27,10 @@ public: enum class Usage { /// starts at 1 for compatibitity with DataTypeEnum8 - None = 1, /// query result neither written nor read into/from query cache - Write, /// query result wrote into query cache - Read, /// query result read from query cache - Unknown, /// we don't know what what happened + Unknown = 1, /// we don't know what what happened + None, /// query result neither written nor read into/from query cache + Write, /// query result written into query cache + Read, /// query result read from query cache }; /// Represents a query result in the cache. diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index c3294512f14..df21e82305a 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -44,10 +44,10 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() auto query_cache_usage_datatype = std::make_shared( DataTypeEnum8::Values { + {"Unknown", static_cast(QueryCache::Usage::Unknown)}, {"None", static_cast(QueryCache::Usage::None)}, {"Write", static_cast(QueryCache::Usage::Write)}, - {"Read", static_cast(QueryCache::Usage::Read)}, - {"Unknown", static_cast(QueryCache::Usage::Unknown)} + {"Read", static_cast(QueryCache::Usage::Read)} }); auto low_cardinality_string = std::make_shared(std::make_shared()); From 043ad45ec3f4f2d1ff8b619da4c6ec5cb2b8f0d0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 09:18:24 +0000 Subject: [PATCH 38/72] Incorporate review feedback, pt. II --- docs/en/operations/system-tables/query_log.md | 8 ++++---- src/Interpreters/Cache/QueryCache.h | 9 ++++----- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 835c79129de..c6f565b8748 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -112,10 +112,10 @@ Columns: - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - - `'Unknown' = 1` = Status unknown. - - `'None' = 2` = The query result was neither written into nor read from the query cache. - - `'Write' = 3` = The query result was written into the query cache. - - `'Read' = 4` = The query result was read from the query cache. + - `'Unknown'` = Status unknown. + - `'None'` = The query result was neither written into nor read from the query cache. + - `'Write'` = The query result was written into the query cache. + - `'Read'` = The query result was read from the query cache. **Example** diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 5fe756268f2..c2de8ca22dd 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -26,11 +26,10 @@ class QueryCache public: enum class Usage { - /// starts at 1 for compatibitity with DataTypeEnum8 - Unknown = 1, /// we don't know what what happened - None, /// query result neither written nor read into/from query cache - Write, /// query result written into query cache - Read, /// query result read from query cache + Unknown, /// we don't know what what happened + None, /// query result neither written nor read into/from query cache + Write, /// query result written into query cache + Read, /// query result read from query cache }; /// Represents a query result in the cache. From 228de12d94a206f6eaae74059216886c32c2b53e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 27 Jul 2023 13:59:23 +0300 Subject: [PATCH 39/72] Update ReplicatedMergeTreeQueue.cpp (#52648) --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index e11913fc3d2..21d5597e614 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1803,7 +1803,7 @@ std::map ReplicatedMergeTreeQueue::getAlterMutationCo Int64 part_data_version = part->info.getDataVersion(); Int64 part_metadata_version = part->getMetadataVersion(); - LOG_DEBUG(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version); + LOG_TEST(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version); std::map result; /// Here we return mutation commands for part which has bigger alter version than part metadata version. From 9d73be6fca63edbca0bd97d07386f41268f1b11d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 12:14:46 +0000 Subject: [PATCH 40/72] test is added --- .../0_stateless/02833_local_udf_options.reference | 1 + tests/queries/0_stateless/02833_local_udf_options.sh | 11 +++++++++++ tests/queries/0_stateless/scripts_udf/function.xml | 9 +++++++++ tests/queries/0_stateless/scripts_udf/udf.sh | 3 +++ 4 files changed, 24 insertions(+) create mode 100755 tests/queries/0_stateless/02833_local_udf_options.reference create mode 100755 tests/queries/0_stateless/02833_local_udf_options.sh create mode 100644 tests/queries/0_stateless/scripts_udf/function.xml create mode 100755 tests/queries/0_stateless/scripts_udf/udf.sh diff --git a/tests/queries/0_stateless/02833_local_udf_options.reference b/tests/queries/0_stateless/02833_local_udf_options.reference new file mode 100755 index 00000000000..19f0805d8de --- /dev/null +++ b/tests/queries/0_stateless/02833_local_udf_options.reference @@ -0,0 +1 @@ +qwerty diff --git a/tests/queries/0_stateless/02833_local_udf_options.sh b/tests/queries/0_stateless/02833_local_udf_options.sh new file mode 100755 index 00000000000..149b62d7e2c --- /dev/null +++ b/tests/queries/0_stateless/02833_local_udf_options.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +SCRIPTS_DIR=$CUR_DIR/scripts_udf + +$CLICKHOUSE_LOCAL -q 'select test_function()' -- --user_scripts_path=$SCRIPTS_DIR --user_defined_executable_functions_config=$SCRIPTS_DIR/function.xml diff --git a/tests/queries/0_stateless/scripts_udf/function.xml b/tests/queries/0_stateless/scripts_udf/function.xml new file mode 100644 index 00000000000..69a0abb5cec --- /dev/null +++ b/tests/queries/0_stateless/scripts_udf/function.xml @@ -0,0 +1,9 @@ + + + executable + test_function + String + TabSeparated + udf.sh + + diff --git a/tests/queries/0_stateless/scripts_udf/udf.sh b/tests/queries/0_stateless/scripts_udf/udf.sh new file mode 100755 index 00000000000..add85833c3e --- /dev/null +++ b/tests/queries/0_stateless/scripts_udf/udf.sh @@ -0,0 +1,3 @@ +#!/bin/sh + +echo qwerty From f3dc6dd061515054afbbe5c58452a9554998a8b7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 27 Jul 2023 15:23:57 +0300 Subject: [PATCH 41/72] Revert "Added field `refcount` to `system.remote_data_paths` table" --- src/Disks/IDisk.h | 5 +- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- .../System/StorageSystemRemoteDataPaths.cpp | 8 +--- .../02791_remote_paths_refcount.reference | 28 ----------- .../02791_remote_paths_refcount.sql | 47 ------------------- 5 files changed, 4 insertions(+), 86 deletions(-) delete mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.reference delete mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.sql diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index a2c5e59237f..2b0ca369a96 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -303,11 +303,10 @@ public: std::string local_path; std::string common_prefix_for_objects; StoredObjects objects; - size_t refcount; LocalPathWithObjectStoragePaths( - const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_, size_t refcount_) - : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)), refcount(refcount_) {} + const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_) + : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)) {} }; virtual void getRemotePathsRecursive(const String &, std::vector &) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 001cff4cefe..762151b3808 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -82,7 +82,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: { try { - paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path), metadata_storage->getHardlinkCount(local_path)); + paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path)); } catch (const Exception & e) { diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 820b1cf3823..eb514d3b3f4 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -25,7 +25,6 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab {"local_path", std::make_shared()}, {"remote_path", std::make_shared()}, {"size", std::make_shared()}, - {"refcount", std::make_shared()}, {"common_prefix_for_blobs", std::make_shared()}, {"cache_paths", std::make_shared(std::make_shared())}, })); @@ -49,7 +48,6 @@ Pipe StorageSystemRemoteDataPaths::read( MutableColumnPtr col_local_path = ColumnString::create(); MutableColumnPtr col_remote_path = ColumnString::create(); MutableColumnPtr col_size = ColumnUInt64::create(); - MutableColumnPtr col_refcount = ColumnUInt64::create(); MutableColumnPtr col_namespace = ColumnString::create(); MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create()); @@ -67,22 +65,19 @@ Pipe StorageSystemRemoteDataPaths::read( if (disk->supportsCache()) cache = FileCacheFactory::instance().getByName(disk->getCacheName()).cache; - for (const auto & [local_path, common_prefox_for_objects, storage_objects, refcount] : remote_paths_by_local_path) + for (const auto & [local_path, common_prefox_for_objects, storage_objects] : remote_paths_by_local_path) { for (const auto & object : storage_objects) { col_disk_name->insert(disk_name); col_base_path->insert(disk->getPath()); - if (cache) col_cache_base_path->insert(cache->getBasePath()); else col_cache_base_path->insertDefault(); - col_local_path->insert(local_path); col_remote_path->insert(object.remote_path); col_size->insert(object.bytes_size); - col_refcount->insert(refcount); col_namespace->insert(common_prefox_for_objects); if (cache) @@ -106,7 +101,6 @@ Pipe StorageSystemRemoteDataPaths::read( res_columns.emplace_back(std::move(col_local_path)); res_columns.emplace_back(std::move(col_remote_path)); res_columns.emplace_back(std::move(col_size)); - res_columns.emplace_back(std::move(col_refcount)); res_columns.emplace_back(std::move(col_namespace)); res_columns.emplace_back(std::move(col_cache_paths)); diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.reference b/tests/queries/0_stateless/02791_remote_paths_refcount.reference deleted file mode 100644 index 56fb1536205..00000000000 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.reference +++ /dev/null @@ -1,28 +0,0 @@ -0_0_0_0 0 -0_0_0_0_1 1 -1_0_0_0 0 -1_0_0_0_1 1 -0_0_0_0_1 checksums.txt 0 -0_0_0_0_1 columns.txt 1 -0_0_0_0_1 count.txt 1 -0_0_0_0_1 default_compression_codec.txt 1 -0_0_0_0_1 id.bin 1 -0_0_0_0_1 id.cmrk2 1 -0_0_0_0_1 metadata_version.txt 1 -0_0_0_0_1 minmax_id.idx 1 -0_0_0_0_1 partition.dat 1 -0_0_0_0_1 primary.cidx 1 -0_0_0_0_1 v.bin 1 -0_0_0_0_1 v.cmrk2 1 -1_0_0_0_1 checksums.txt 0 -1_0_0_0_1 columns.txt 0 -1_0_0_0_1 count.txt 1 -1_0_0_0_1 default_compression_codec.txt 0 -1_0_0_0_1 id.bin 1 -1_0_0_0_1 id.cmrk2 1 -1_0_0_0_1 metadata_version.txt 0 -1_0_0_0_1 minmax_id.idx 1 -1_0_0_0_1 partition.dat 1 -1_0_0_0_1 primary.cidx 1 -1_0_0_0_1 v.bin 0 -1_0_0_0_1 v.cmrk2 0 diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql deleted file mode 100644 index 180601738ad..00000000000 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.sql +++ /dev/null @@ -1,47 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS t_refcount SYNC; - --- Names of parts (on which this test depends) --- can differ in case of fault injection. -SET insert_keeper_fault_injection_probability = 0.0; - -CREATE TABLE t_refcount (id UInt64, v UInt64) -ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') -ORDER BY id PARTITION BY id % 2 -SETTINGS - storage_policy = 's3_cache', - allow_remote_fs_zero_copy_replication = 1, - min_bytes_for_wide_part = 0, - compress_marks = 1, - compress_primary_key = 1, - ratio_of_defaults_for_sparse_serialization = 1.0; - -INSERT INTO t_refcount VALUES (1, 10), (2, 20); - -SET mutations_sync = 2; -ALTER TABLE t_refcount UPDATE v = v * 10 WHERE id % 2 = 1; - -SELECT name, active FROM system.parts WHERE database = currentDatabase() AND table = 't_refcount' ORDER BY name; - -WITH splitByChar('/', full_path) AS path_parts -SELECT path_parts[-2] AS part_name, path_parts[-1] AS file_name, refcount -FROM -( - SELECT - path || local_path AS full_path, - substring(full_path, 1, length(full_path) - position(reverse(full_path), '/') + 1) AS part_path, - refcount - FROM system.remote_data_paths - WHERE disk_name = 's3_cache' -) AS paths -INNER JOIN -( - SELECT path - FROM system.parts - WHERE database = currentDatabase() AND table = 't_refcount' AND active -) AS parts -ON paths.part_path = parts.path -ORDER BY part_name, file_name; - -DROP TABLE IF EXISTS t_refcount SYNC; From 8bba7baeaa65548d91da4c068c6af9b583f9449c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 12:34:22 +0000 Subject: [PATCH 42/72] fix style --- tests/queries/0_stateless/02833_local_udf_options.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02833_local_udf_options.reference diff --git a/tests/queries/0_stateless/02833_local_udf_options.reference b/tests/queries/0_stateless/02833_local_udf_options.reference old mode 100755 new mode 100644 From 33300a978e00687713d08fa786178eecc7bc15d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 15:38:53 +0300 Subject: [PATCH 43/72] Update CHANGELOG (#52655) --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 878edfa4add..f401b346726 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,7 +20,7 @@ * The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* Added support for PRQL as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). * Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). * Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). * Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). @@ -115,6 +115,7 @@ * Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). #### Experimental Feature +* Added support for [PRQL](https://prql-lang.org/) as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). * Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). This syntax can be changed in this release. * (experimental MaterializedMySQL) Fixed crash when `mysqlxx::Pool::Entry` is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). * (experimental MaterializedMySQL) `CREATE TABLE ... AS SELECT` .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). From 78f3a575f9ddbfd47e46e8169b63979e3d2aa72f Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Thu, 27 Jul 2023 21:06:34 +0800 Subject: [PATCH 44/72] Convert hashSets in parallel before merge (#50748) * Convert hashSets in parallel before merge Before merge, if one of the lhs and rhs is singleLevelSet and the other is twoLevelSet, then the SingleLevelSet will call convertToTwoLevel(). The convert process is not in parallel and it will cost lots of cycle if it cosume all the singleLevelSet. The idea of the patch is to convert all the singleLevelSets to twoLevelSets in parallel if the hashsets are not all singleLevel or not all twoLevel. I have tested the patch on Intel 2 x 112 vCPUs SPR server with clickbench and latest upstream ClickHouse. Q5 has got a big 264% performance improvement and 24 queries have got at least 5% performance gain. The overall geomean of 43 queries has gained 7.4% more than the base code. Signed-off-by: Jiebin Sun * add resize() for the data_vec in parallelizeMergePrepare() Signed-off-by: Jiebin Sun * Add the performance test prepare_hash_before_merge.xml Signed-off-by: Jiebin Sun * Fit the CI to rename the data set from hits_v1 to test.hits. Signed-off-by: Jiebin Sun * remove the redundant branch in UniqExactSet Co-authored-by: Nikita Taranov * Remove the empty methods and add throw exception in parallelizeMergePrepare() Signed-off-by: Jiebin Sun --------- Signed-off-by: Jiebin Sun Co-authored-by: Nikita Taranov --- .../AggregateFunctionUniq.h | 39 ++++++++++++++ src/AggregateFunctions/IAggregateFunction.h | 8 +++ src/AggregateFunctions/UniqExactSet.h | 51 +++++++++++++++++++ src/Interpreters/Aggregator.cpp | 14 +++++ .../performance/prepare_hash_before_merge.xml | 4 ++ 5 files changed, 116 insertions(+) create mode 100644 tests/performance/prepare_hash_before_merge.xml diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index de68e9076a0..2810051a82f 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -29,6 +29,10 @@ #include #include +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} namespace DB { @@ -42,6 +46,7 @@ struct AggregateFunctionUniqUniquesHashSetData Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniq"; } @@ -55,6 +60,7 @@ struct AggregateFunctionUniqUniquesHashSetDataForVariadic Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -72,6 +78,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -84,6 +91,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -96,6 +104,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -108,6 +117,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -120,6 +130,7 @@ struct AggregateFunctionUniqHLL12DataForVariadic Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -143,6 +154,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -162,6 +174,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -181,6 +194,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -190,6 +204,7 @@ template { constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -204,6 +219,7 @@ struct AggregateFunctionUniqThetaData Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqTheta"; } @@ -213,6 +229,7 @@ template struct AggregateFunctionUniqThetaDataForVariadic : AggregateFunctionUniqThetaData { constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -384,8 +401,10 @@ template class AggregateFunctionUniq final : public IAggregateFunctionDataHelper> { private: + using DataSet = typename Data::Set; static constexpr size_t num_args = 1; static constexpr bool is_able_to_parallelize_merge = Data::is_able_to_parallelize_merge; + static constexpr bool is_parallelize_merge_prepare_needed = Data::is_parallelize_merge_prepare_needed; public: explicit AggregateFunctionUniq(const DataTypes & argument_types_) @@ -439,6 +458,26 @@ public: detail::Adder::add(this->data(place), columns, num_args, row_begin, row_end, flags, null_map); } + bool isParallelizeMergePrepareNeeded() const override { return is_parallelize_merge_prepare_needed;} + + void parallelizeMergePrepare(AggregateDataPtrs & places, ThreadPool & thread_pool) const override + { + if constexpr (is_parallelize_merge_prepare_needed) + { + std::vector data_vec; + data_vec.resize(places.size()); + + for (unsigned long i = 0; i < data_vec.size(); i++) + data_vec[i] = &this->data(places[i]).set; + + DataSet::parallelizeMergePrepare(data_vec, thread_pool); + } + else + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "parallelizeMergePrepare() is only implemented when is_parallelize_merge_prepare_needed is true for {} ", getName()); + } + } + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).set.merge(this->data(rhs).set); diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index df08b6f2109..b460a66ea22 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -47,6 +47,7 @@ using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; using AggregateDataPtr = char *; +using AggregateDataPtrs = std::vector; using ConstAggregateDataPtr = const char *; class IAggregateFunction; @@ -148,6 +149,13 @@ public: /// Default values must be a the 0-th positions in columns. virtual void addManyDefaults(AggregateDataPtr __restrict place, const IColumn ** columns, size_t length, Arena * arena) const = 0; + virtual bool isParallelizeMergePrepareNeeded() const { return false; } + + virtual void parallelizeMergePrepare(AggregateDataPtrs & /*places*/, ThreadPool & /*thread_pool*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "parallelizeMergePrepare() with thread pool parameter isn't implemented for {} ", getName()); + } + /// Merges state (on which place points to) with other state of current aggregation function. virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const = 0; diff --git a/src/AggregateFunctions/UniqExactSet.h b/src/AggregateFunctions/UniqExactSet.h index 90cfe700179..0d99b29686f 100644 --- a/src/AggregateFunctions/UniqExactSet.h +++ b/src/AggregateFunctions/UniqExactSet.h @@ -28,6 +28,57 @@ public: asTwoLevel().insert(std::forward(arg)); } + /// In merge, if one of the lhs and rhs is twolevelset and the other is singlelevelset, then the singlelevelset will need to convertToTwoLevel(). + /// It's not in parallel and will cost extra large time if the thread_num is large. + /// This method will convert all the SingleLevelSet to TwoLevelSet in parallel if the hashsets are not all singlelevel or not all twolevel. + static void parallelizeMergePrepare(const std::vector & data_vec, ThreadPool & thread_pool) + { + unsigned long single_level_set_num = 0; + + for (auto ele : data_vec) + { + if (ele->isSingleLevel()) + single_level_set_num ++; + } + + if (single_level_set_num > 0 && single_level_set_num < data_vec.size()) + { + try + { + auto data_vec_atomic_index = std::make_shared(0); + auto thread_func = [data_vec, data_vec_atomic_index, thread_group = CurrentThread::getGroup()]() + { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + + setThreadName("UniqExaConvert"); + + while (true) + { + const auto i = data_vec_atomic_index->fetch_add(1); + if (i >= data_vec.size()) + return; + if (data_vec[i]->isSingleLevel()) + data_vec[i]->convertToTwoLevel(); + } + }; + for (size_t i = 0; i < std::min(thread_pool.getMaxThreads(), single_level_set_num); ++i) + thread_pool.scheduleOrThrowOnError(thread_func); + + thread_pool.wait(); + } + catch (...) + { + thread_pool.wait(); + throw; + } + } + } + auto merge(const UniqExactSet & other, ThreadPool * thread_pool = nullptr) { if (isSingleLevel() && other.isTwoLevel()) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 36cd32910b5..c2914c938b5 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2603,6 +2603,20 @@ void NO_INLINE Aggregator::mergeWithoutKeyDataImpl( AggregatedDataVariantsPtr & res = non_empty_data[0]; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (aggregate_functions[i]->isParallelizeMergePrepareNeeded()) + { + size_t size = non_empty_data.size(); + std::vector data_vec; + + for (size_t result_num = 0; result_num < size; ++result_num) + data_vec.emplace_back(non_empty_data[result_num]->without_key + offsets_of_aggregate_states[i]); + + aggregate_functions[i]->parallelizeMergePrepare(data_vec, thread_pool); + } + } + /// We merge all aggregation results to the first. for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) { diff --git a/tests/performance/prepare_hash_before_merge.xml b/tests/performance/prepare_hash_before_merge.xml new file mode 100644 index 00000000000..e99f762927f --- /dev/null +++ b/tests/performance/prepare_hash_before_merge.xml @@ -0,0 +1,4 @@ + + SELECT COUNT(DISTINCT Title) FROM test.hits SETTINGS max_threads = 24 + SELECT COUNT(DISTINCT Referer) FROM test.hits SETTINGS max_threads = 22 + From 671128140dc6672349421c84643076410d46ce0f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 15:34:05 +0200 Subject: [PATCH 45/72] Update autogenerated version to 23.8.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 ++++---- .../StorageSystemContributors.generated.cpp | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 821b7b46855..9919d018046 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54476) +SET(VERSION_REVISION 54477) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 7) +SET(VERSION_MINOR 8) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH d1c7e13d08868cb04d3562dcced704dd577cb1df) -SET(VERSION_DESCRIBE v23.7.1.1-testing) -SET(VERSION_STRING 23.7.1.1) +SET(VERSION_GITHASH a70127baecc451f1f7073bad7b6198f6703441d8) +SET(VERSION_DESCRIBE v23.8.1.1-testing) +SET(VERSION_STRING 23.8.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index f84c554afc0..031c7454ab6 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -42,10 +42,12 @@ const char * auto_contributors[] { "Alex", "Alex Bocharov", "Alex Cao", + "Alex Cheng", "Alex Karo", "Alex Krash", "Alex Ryndin", "Alex Zatelepin", + "AlexBykovski", "Alexander Avdonkin", "Alexander Bezpiatov", "Alexander Burmak", @@ -232,6 +234,7 @@ const char * auto_contributors[] { "CheSema", "Chebarykov Pavel", "Chen Yufei", + "Chen768959", "Cheng Pan", "Chienlung Cheung", "Christian", @@ -485,6 +488,7 @@ const char * auto_contributors[] { "John", "John Hummel", "John Skopis", + "John Spurlock", "Jonatas Freitas", "Jonathan-Ackerman", "Jordi", @@ -659,6 +663,7 @@ const char * auto_contributors[] { "Mikhail Gaidamaka", "Mikhail Guzov", "Mikhail Korotov", + "Mikhail Koviazin", "Mikhail Malafeev", "Mikhail Nacharov", "Mikhail Salosin", @@ -815,6 +820,7 @@ const char * auto_contributors[] { "Roman Vasin", "Roman Vlasenko", "Roman Zhukov", + "Rory Crispin", "Roy Bellingan", "Ruslan", "Ruslan Savchenko", @@ -832,7 +838,9 @@ const char * auto_contributors[] { "Salvatore Mesoraca", "Sami Kerola", "Samuel Chou", + "Samuel Colvin", "San", + "Sanjam Panda", "Saulius Valatka", "Sean Haynes", "Sean Lafferty", @@ -883,6 +891,7 @@ const char * auto_contributors[] { "SmitaRKulkarni", "Snow", "Sofia Antipushina", + "Song Liyong", "Sorck", "Stanislav Dobrovolschii", "Stanislav Pavlovichev", @@ -893,6 +902,7 @@ const char * auto_contributors[] { "Stepan Herold", "Stephan", "Steve-金勇", + "StianBerger", "Stig Bakken", "Storozhuk Kostiantyn", "Stupnikov Andrey", @@ -977,6 +987,7 @@ const char * auto_contributors[] { "Vitaliy Karnienko", "Vitaliy Kozlovskiy", "Vitaliy Lyudvichenko", + "Vitaliy Pashkov", "Vitaliy Zakaznikov", "Vitaly", "Vitaly Artemyev", @@ -1029,6 +1040,7 @@ const char * auto_contributors[] { "Yakov Olkhovskiy", "YalalovSM", "Yangkuan Liu", + "Yarik Briukhovetskyi", "Yatian Xu", "Yatsishin Ilya", "Yağızcan Değirmenci", @@ -1053,6 +1065,7 @@ const char * auto_contributors[] { "Yury Karpovich", "Yury Stankevich", "Yusuke Tanaka", + "Zach Naimon", "ZhiYong Wang", "Zhichang Yu", "Zhichun Wu", @@ -1143,6 +1156,7 @@ const char * auto_contributors[] { "changvvb", "chasingegg", "chen", + "chen768959", "chen9t", "chengy8934", "chenjian", @@ -1179,6 +1193,7 @@ const char * auto_contributors[] { "detailyang", "dfenelonov", "dgrr", + "dheerajathrey", "dimarub2000", "dinosaur", "divanorama", @@ -1329,6 +1344,7 @@ const char * auto_contributors[] { "lanfz", "larryluogit", "laurieliyang", + "lcjh", "lehasm", "leosunli", "leozhang", @@ -1455,6 +1471,7 @@ const char * auto_contributors[] { "pawelsz-rb", "pdai", "pdv-ru", + "pedro.riera", "pengxiangcai", "peshkurov", "peter279k", @@ -1548,8 +1565,10 @@ const char * auto_contributors[] { "teng.ma", "terrylin", "tesw yew isal", + "therealnick233", "tianzhou", "tiger.yan", + "timfursov", "tison", "topvisor", "tpanetti", @@ -1563,6 +1582,7 @@ const char * auto_contributors[] { "usurai", "vahid-sohrabloo", "vdimir", + "velavokr", "velom", "vesslanjin", "vgocoder", @@ -1587,17 +1607,21 @@ const char * auto_contributors[] { "wuxiaobai24", "wzl", "xPoSx", + "xiao", + "xiaolei565", "xiedeyantu", "xieyichen", "xinhuitian", "xlwh", "xmy", + "xuelei", "yakkomajuri", "yakov-olkhovskiy", "yandd", "yang", "yangshuai", "yaqi-zhao", + "yariks5s", "yeer", "ygrek", "yhgcn", From 28c49e3f20923917c46872b15dce21a72ab47b4a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Jul 2023 13:47:11 +0000 Subject: [PATCH 46/72] Update version_date.tsv and changelogs after v23.7.1.2470-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.7.1.2470-stable.md | 452 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 458 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v23.7.1.2470-stable.md diff --git a/SECURITY.md b/SECURITY.md index 4ba5f13d09c..d61533b44b9 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.7 | ✔️ | | 23.6 | ✔️ | | 23.5 | ✔️ | -| 23.4 | ✔️ | +| 23.4 | ❌ | | 23.3 | ✔️ | | 23.2 | ❌ | | 23.1 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 8a6324aef88..c9800e4e66d 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 7f453627601..f558338b23c 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 1fa7b83ae16..156de034a7f 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.7.1.2470-stable.md b/docs/changelogs/v23.7.1.2470-stable.md new file mode 100644 index 00000000000..a77078cb653 --- /dev/null +++ b/docs/changelogs/v23.7.1.2470-stable.md @@ -0,0 +1,452 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.7.1.2470-stable (a70127baecc) FIXME as compared to v23.6.1.1524-stable (d1c7e13d088) + +#### Backward Incompatible Change +* Add ` NAMED COLLECTION` access type (aliases `USE NAMED COLLECTION`, `NAMED COLLECTION USAGE`). This PR is backward incompatible because this access type is disabled by default (because a parent access type `NAMED COLLECTION ADMIN` is disabled by default as well). Proposed in [#50277](https://github.com/ClickHouse/ClickHouse/issues/50277). To grant use `GRANT NAMED COLLECTION ON collection_name TO user` or `GRANT NAMED COLLECTION ON * TO user`, to be able to give these grants `named_collection_admin` is required in config (previously it was named `named_collection_control`, so will remain as an alias). [#50625](https://github.com/ClickHouse/ClickHouse/pull/50625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixing a typo in the `system.parts` column name `last_removal_attemp_time`. Now it is named `last_removal_attempt_time`. [#52104](https://github.com/ClickHouse/ClickHouse/pull/52104) ([filimonov](https://github.com/filimonov)). +* Bump version of the distributed_ddl_entry_format_version to 5 by default (enables opentelemetry and initial_query_idd pass through). This will not allow to process existing entries for distributed DDL after **downgrade** (but note, that usually there should be no such unprocessed entries). [#52128](https://github.com/ClickHouse/ClickHouse/pull/52128) ([Azat Khuzhin](https://github.com/azat)). +* Check projection metadata the same way we check ordinary metadata. This change may prevent the server from starting in case there was a table with an invalid projection. An example is a projection that created positional columns in PK (e.g. `projection p (select * order by 1, 4)` which is not allowed in table PK and can cause a crash during insert/merge). Drop such projections before the update. Fixes [#52353](https://github.com/ClickHouse/ClickHouse/issues/52353). [#52361](https://github.com/ClickHouse/ClickHouse/pull/52361) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The function `toDecimalString` is removed due to subpar implementation quality. This closes [#52407](https://github.com/ClickHouse/ClickHouse/issues/52407). [#52450](https://github.com/ClickHouse/ClickHouse/pull/52450) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Implement KQL-style formatting for Interval. [#45671](https://github.com/ClickHouse/ClickHouse/pull/45671) ([ltrk2](https://github.com/ltrk2)). +* Support ZooKeeper `reconfig` command for CH Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). +* Kafka connector can fetch avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). +* Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Added support for prql as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add a column is_obsolete to system.settings and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). +* Implement support of encrypted elements in configuration file Added possibility to use encrypted text in leaf elements of configuration file. The text is encrypted using encryption codecs from section. [#50986](https://github.com/ClickHouse/ClickHouse/pull/50986) ([Roman Vasin](https://github.com/rvasin)). +* Just a new request of [#49483](https://github.com/ClickHouse/ClickHouse/issues/49483). [#51013](https://github.com/ClickHouse/ClickHouse/pull/51013) ([lgbo](https://github.com/lgbo-ustc)). +* Add SYSTEM STOP LISTEN query. Closes [#47972](https://github.com/ClickHouse/ClickHouse/issues/47972). [#51016](https://github.com/ClickHouse/ClickHouse/pull/51016) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add input_format_csv_allow_variable_number_of_columns options. [#51273](https://github.com/ClickHouse/ClickHouse/pull/51273) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Another boring feature: add function substring_index, as in spark or mysql. [#51472](https://github.com/ClickHouse/ClickHouse/pull/51472) ([李扬](https://github.com/taiyang-li)). +* Show stats for jemalloc bins. Example ``` SELECT *, size * (nmalloc - ndalloc) AS allocated_bytes FROM system.jemalloc_bins WHERE allocated_bytes > 0 ORDER BY allocated_bytes DESC LIMIT 10. [#51674](https://github.com/ClickHouse/ClickHouse/pull/51674) ([Alexander Gololobov](https://github.com/davenger)). +* Add RowBinaryWithDefaults format with extra byte before each column for using column default value. Closes [#50854](https://github.com/ClickHouse/ClickHouse/issues/50854). [#51695](https://github.com/ClickHouse/ClickHouse/pull/51695) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `default_temporary_table_engine` setting. Same as `default_table_engine` but for temporary tables. [#51292](https://github.com/ClickHouse/ClickHouse/issues/51292). [#51708](https://github.com/ClickHouse/ClickHouse/pull/51708) ([velavokr](https://github.com/velavokr)). +* Added new initcap / initcapUTF8 functions which convert the first letter of each word to upper case and the rest to lower case. [#51735](https://github.com/ClickHouse/ClickHouse/pull/51735) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create table now supports `PRIMARY KEY` syntax in column definition. Columns are added to primary index in the same order columns are defined. [#51881](https://github.com/ClickHouse/ClickHouse/pull/51881) ([Ilya Yatsishin](https://github.com/qoega)). +* Added the possibility to use date and time format specifiers in log and error log file names, either in config files (`log` and `errorlog` tags) or command line arguments (`--log-file` and `--errorlog-file`). [#51945](https://github.com/ClickHouse/ClickHouse/pull/51945) ([Victor Krasnov](https://github.com/sirvickr)). +* Added Peak Memory Usage (for query) to client final statistics, and to http header. [#51946](https://github.com/ClickHouse/ClickHouse/pull/51946) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added new hasSubsequence() (+CaseInsensitive + UTF8 versions) functions. [#52050](https://github.com/ClickHouse/ClickHouse/pull/52050) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Add `array_agg` as alias of `groupArray` for PostgreSQL compatibility. Closes [#52100](https://github.com/ClickHouse/ClickHouse/issues/52100). ### Documentation entry for user-facing changes. [#52135](https://github.com/ClickHouse/ClickHouse/pull/52135) ([flynn](https://github.com/ucasfl)). +* Add `any_value` as a compatibility alias for `any` aggregate function. Closes [#52140](https://github.com/ClickHouse/ClickHouse/issues/52140). [#52147](https://github.com/ClickHouse/ClickHouse/pull/52147) ([flynn](https://github.com/ucasfl)). +* Add aggregate function `array_concat_agg` for compatibility with BigQuery, it's alias of `groupArrayArray`. Closes [#52139](https://github.com/ClickHouse/ClickHouse/issues/52139). [#52149](https://github.com/ClickHouse/ClickHouse/pull/52149) ([flynn](https://github.com/ucasfl)). +* Add `OCTET_LENGTH` as an alias to `length`. Closes [#52153](https://github.com/ClickHouse/ClickHouse/issues/52153). [#52176](https://github.com/ClickHouse/ClickHouse/pull/52176) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Re-add SipHash keyed functions. [#52206](https://github.com/ClickHouse/ClickHouse/pull/52206) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Added `firstLine` function to extract the first line from the multi-line string. This closes [#51172](https://github.com/ClickHouse/ClickHouse/issues/51172). [#52209](https://github.com/ClickHouse/ClickHouse/pull/52209) ([Mikhail Koviazin](https://github.com/mkmkme)). + +#### Performance Improvement +* Enable `move_all_conditions_to_prewhere` and `enable_multiple_prewhere_read_steps` settings by default. [#46365](https://github.com/ClickHouse/ClickHouse/pull/46365) ([Alexander Gololobov](https://github.com/davenger)). +* Improves performance of some queries by tuning allocator. [#46416](https://github.com/ClickHouse/ClickHouse/pull/46416) ([Azat Khuzhin](https://github.com/azat)). +* Writing parquet files is 10x faster, it's multi-threaded now. Almost the same speed as reading. [#49367](https://github.com/ClickHouse/ClickHouse/pull/49367) ([Michael Kolupaev](https://github.com/al13n321)). +* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Now we use fixed-size tasks in `MergeTreePrefetchedReadPool` as in `MergeTreeReadPool`. Also from now we use connection pool for S3 requests. [#49732](https://github.com/ClickHouse/ClickHouse/pull/49732) ([Nikita Taranov](https://github.com/nickitat)). +* More pushdown to the right side of join. [#50532](https://github.com/ClickHouse/ClickHouse/pull/50532) ([Nikita Taranov](https://github.com/nickitat)). +* Improve grace_hash join by reserving hash table's size (resubmit). [#50875](https://github.com/ClickHouse/ClickHouse/pull/50875) ([lgbo](https://github.com/lgbo-ustc)). +* Waiting on lock in `OpenedFileCache` could be noticeable sometimes. We sharded it into multiple sub-maps (each with its own lock) to avoid contention. [#51341](https://github.com/ClickHouse/ClickHouse/pull/51341) ([Nikita Taranov](https://github.com/nickitat)). +* Remove duplicate condition in functionunixtimestamp64.h. [#51857](https://github.com/ClickHouse/ClickHouse/pull/51857) ([lcjh](https://github.com/ljhcage)). +* The idea is that conditions with PK columns are likely to be used in PK analysis and will not contribute much more to PREWHERE filtering. [#51958](https://github.com/ClickHouse/ClickHouse/pull/51958) ([Alexander Gololobov](https://github.com/davenger)). +* 1. Add rewriter for both old and new analyzer. 2. Add settings `optimize_uniq_to_count` which default is 0. [#52004](https://github.com/ClickHouse/ClickHouse/pull/52004) ([JackyWoo](https://github.com/JackyWoo)). +* The performance experiments of **OnTime** on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) show that this change could bring an improvement of **11.6%** to the QPS of the query **Q8** while having no impact on others. [#52036](https://github.com/ClickHouse/ClickHouse/pull/52036) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Enable `allow_vertical_merges_from_compact_to_wide_parts` by default. It will save memory usage during merges. [#52295](https://github.com/ClickHouse/ClickHouse/pull/52295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis which invalidates primary keys. This issue only exists when `query_plan_optimize_primary_key = 1, query_plan_optimize_projection = 1` . This fixes [#48823](https://github.com/ClickHouse/ClickHouse/issues/48823) . This fixes [#51173](https://github.com/ClickHouse/ClickHouse/issues/51173) . [#52308](https://github.com/ClickHouse/ClickHouse/pull/52308) ([Amos Bird](https://github.com/amosbird)). +* Reduce the number of syscalls in FileCache::loadMetadata. [#52435](https://github.com/ClickHouse/ClickHouse/pull/52435) ([Raúl Marín](https://github.com/Algunenano)). + +#### Improvement +* Added query `SYSTEM FLUSH ASYNC INSERT QUEUE` which flushes all pending asynchronous inserts to the destination tables. Added a server-side setting `async_insert_queue_flush_on_shutdown` (`true` by default) which determines whether to flush queue of asynchronous inserts on graceful shutdown. Setting `async_insert_threads` is now a server-side setting. [#49160](https://github.com/ClickHouse/ClickHouse/pull/49160) ([Anton Popov](https://github.com/CurtizJ)). +* Don't show messages about `16 EiB` free space in logs, as they don't make sense. This closes [#49320](https://github.com/ClickHouse/ClickHouse/issues/49320). [#49342](https://github.com/ClickHouse/ClickHouse/pull/49342) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Properly check the limit for the `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block`. This is needed for generic query fuzzer. [#49343](https://github.com/ClickHouse/ClickHouse/pull/49343) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix two issues: ``` select geohashEncode(120.2, number::Float64) from numbers(10);. [#50066](https://github.com/ClickHouse/ClickHouse/pull/50066) ([李扬](https://github.com/taiyang-li)). +* Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). +* Allow to have strict lower boundary for file segment size by downloading remaining data in the background. Minimum size of file segment (if actual file size is bigger) is configured as cache configuration setting `boundary_alignment`, by default `4Mi`. Number of background threads are configured as cache configuration setting `background_download_threads`, by default `2`. Also `max_file_segment_size` was increased from `8Mi` to `32Mi` in this PR. [#51000](https://github.com/ClickHouse/ClickHouse/pull/51000) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow filtering HTTP headers with `http_forbid_headers` section in config. Both exact matching and regexp filters are available. [#51038](https://github.com/ClickHouse/ClickHouse/pull/51038) ([Nikolay Degterinsky](https://github.com/evillique)). +* #50727 new alias for function current_database and added new function current_schemas. [#51076](https://github.com/ClickHouse/ClickHouse/pull/51076) ([Pedro Riera](https://github.com/priera)). +* Log async insert flush queries into to system.query_log. [#51160](https://github.com/ClickHouse/ClickHouse/pull/51160) ([Raúl Marín](https://github.com/Algunenano)). +* Decreased default timeouts for S3 from 30 seconds to 3 seconds, and for other HTTP from 180 seconds to 30 seconds. [#51171](https://github.com/ClickHouse/ClickHouse/pull/51171) ([Michael Kolupaev](https://github.com/al13n321)). +* Use read_bytes/total_bytes_to_read for progress bar in s3/file/url/... table functions for better progress indication. [#51286](https://github.com/ClickHouse/ClickHouse/pull/51286) ([Kruglov Pavel](https://github.com/Avogar)). +* Functions "date_diff() and age()" now support millisecond/microsecond unit and work with microsecond precision. [#51291](https://github.com/ClickHouse/ClickHouse/pull/51291) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Allow SQL standard `FETCH` without `OFFSET`. See https://antonz.org/sql-fetch/. [#51293](https://github.com/ClickHouse/ClickHouse/pull/51293) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve parsing of path in clickhouse-keeper-client. [#51359](https://github.com/ClickHouse/ClickHouse/pull/51359) ([Azat Khuzhin](https://github.com/azat)). +* A third-party product depending on ClickHouse (Gluten: Plugin to Double SparkSQL's Performance) had a bug. This fix avoids heap overflow in that third-party product while reading from HDFS. [#51386](https://github.com/ClickHouse/ClickHouse/pull/51386) ([李扬](https://github.com/taiyang-li)). +* Fix checking error caused by uninitialized class members. [#51418](https://github.com/ClickHouse/ClickHouse/pull/51418) ([李扬](https://github.com/taiyang-li)). +* Add ability to disable native copy for S3 (setting for BACKUP/RESTORE `allow_s3_native_copy`, and `s3_allow_native_copy` for `s3`/`s3_plain` disks). [#51448](https://github.com/ClickHouse/ClickHouse/pull/51448) ([Azat Khuzhin](https://github.com/azat)). +* Add column `primary_key_size` to `system.parts` table to show compressed primary key size on disk. Closes [#51400](https://github.com/ClickHouse/ClickHouse/issues/51400). [#51496](https://github.com/ClickHouse/ClickHouse/pull/51496) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Allow running `clickhouse-local` without procfs, without home directory existing, and without name resolution plugins from glibc. [#51518](https://github.com/ClickHouse/ClickHouse/pull/51518) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Correcting the message of modify storage policy https://github.com/clickhouse/clickhouse/issues/51516 ### documentation entry for user-facing changes. [#51519](https://github.com/ClickHouse/ClickHouse/pull/51519) ([xiaolei565](https://github.com/xiaolei565)). +* Support `DROP FILESYSTEM CACHE KEY [ OFFSET ]`. [#51547](https://github.com/ClickHouse/ClickHouse/pull/51547) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add placeholder `%a` for rull filename in rename_files_after_processing setting. [#51603](https://github.com/ClickHouse/ClickHouse/pull/51603) ([Kruglov Pavel](https://github.com/Avogar)). +* Add column modification time into system.parts_columns. [#51685](https://github.com/ClickHouse/ClickHouse/pull/51685) ([Azat Khuzhin](https://github.com/azat)). +* Add new setting `input_format_csv_use_default_on_bad_values` to CSV format that allows to insert default value when parsing of a single field failed. [#51716](https://github.com/ClickHouse/ClickHouse/pull/51716) ([KevinyhZou](https://github.com/KevinyhZou)). +* Added a crash log flush to the disk after the unexpected crash. [#51720](https://github.com/ClickHouse/ClickHouse/pull/51720) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix behavior in dashboard page where errors unrelated to authentication are not shown. Also fix 'overlapping' chart behavior. [#51744](https://github.com/ClickHouse/ClickHouse/pull/51744) ([Zach Naimon](https://github.com/ArctypeZach)). +* Allow UUID to UInt128 conversion. [#51765](https://github.com/ClickHouse/ClickHouse/pull/51765) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added support for function range of Nullable arguments. [#51767](https://github.com/ClickHouse/ClickHouse/pull/51767) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Convert condition like `toyear(x) = c` to `c1 <= x < c2`. [#51795](https://github.com/ClickHouse/ClickHouse/pull/51795) ([Han Fei](https://github.com/hanfei1991)). +* Improve MySQL compatibility of statement SHOW INDEX. [#51796](https://github.com/ClickHouse/ClickHouse/pull/51796) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix `use_structure_from_insertion_table_in_table_functions` does not work with `MATERIALIZED` and `ALIAS` columns. Closes [#51817](https://github.com/ClickHouse/ClickHouse/issues/51817). Closes [#51019](https://github.com/ClickHouse/ClickHouse/issues/51019). [#51825](https://github.com/ClickHouse/ClickHouse/pull/51825) ([flynn](https://github.com/ucasfl)). +* Introduce a table setting `wait_for_unique_parts_send_before_shutdown_ms` which specify the amount of time replica will wait before closing interserver handler for replicated sends. Also fix inconsistency with shutdown of tables and interserver handlers: now server shutdown tables first and only after it shut down interserver handlers. [#51851](https://github.com/ClickHouse/ClickHouse/pull/51851) ([alesapin](https://github.com/alesapin)). +* CacheDictionary request only unique keys from source. Closes [#51762](https://github.com/ClickHouse/ClickHouse/issues/51762). [#51853](https://github.com/ClickHouse/ClickHouse/pull/51853) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed settings not applied for explain query when format provided. [#51859](https://github.com/ClickHouse/ClickHouse/pull/51859) ([Nikita Taranov](https://github.com/nickitat)). +* Allow SETTINGS before FORMAT in DESCRIBE TABLE query for compatibility with SELECT query. Closes [#51544](https://github.com/ClickHouse/ClickHouse/issues/51544). [#51899](https://github.com/ClickHouse/ClickHouse/pull/51899) ([Nikolay Degterinsky](https://github.com/evillique)). +* Var-int encoded integers (e.g. used by the native protocol) can now use the full 64-bit range. 3rd party clients are advised to update their var-int code accordingly. [#51905](https://github.com/ClickHouse/ClickHouse/pull/51905) ([Robert Schulze](https://github.com/rschu1ze)). +* Update certificates when they change without the need to manually SYSTEM RELOAD CONFIG. [#52030](https://github.com/ClickHouse/ClickHouse/pull/52030) ([Mike Kot](https://github.com/myrrc)). +* Added `allow_create_index_without_type` setting that allow to ignore `ADD INDEX` queries without specified `TYPE`. Standard SQL queries will just succeed without changing table schema. [#52056](https://github.com/ClickHouse/ClickHouse/pull/52056) ([Ilya Yatsishin](https://github.com/qoega)). +* Fixed crash when mysqlxx::Pool::Entry is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). +* CREATE TABLE ... AS SELECT .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). +* Introduced automatic conversion of text types to utf8 for MaterializedMySQL. [#52084](https://github.com/ClickHouse/ClickHouse/pull/52084) ([Val Doroshchuk](https://github.com/valbok)). +* Add alias for functions `today` (now available under the `curdate`/`current_date` names) and `now` (`current_timestamp`). [#52106](https://github.com/ClickHouse/ClickHouse/pull/52106) ([Lloyd-Pottiger](https://github.com/Lloyd-Pottiger)). +* Log messages are written to text_log from the beginning. [#52113](https://github.com/ClickHouse/ClickHouse/pull/52113) ([Dmitry Kardymon](https://github.com/kardymonds)). +* In cases where the HTTP endpoint has multiple IP addresses and the first of them is unreachable, a timeout exception will be thrown. Made session creation with handling all resolved endpoints. [#52116](https://github.com/ClickHouse/ClickHouse/pull/52116) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Support async_deduplication_token for async insert. [#52136](https://github.com/ClickHouse/ClickHouse/pull/52136) ([Han Fei](https://github.com/hanfei1991)). +* Avro input format support Union with single type. Closes [#52131](https://github.com/ClickHouse/ClickHouse/issues/52131). [#52137](https://github.com/ClickHouse/ClickHouse/pull/52137) ([flynn](https://github.com/ucasfl)). +* Add setting `optimize_use_implicit_projections` to disable implicit projections (currently only `min_max_count` projection). This is defaulted to false until [#52075](https://github.com/ClickHouse/ClickHouse/issues/52075) is fixed. [#52152](https://github.com/ClickHouse/ClickHouse/pull/52152) ([Amos Bird](https://github.com/amosbird)). +* It was possible to use the function `hasToken` for infinite loop. Now this possibility is removed. This closes [#52156](https://github.com/ClickHouse/ClickHouse/issues/52156). [#52160](https://github.com/ClickHouse/ClickHouse/pull/52160) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* 1. Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). +* Functions "date_diff() and age()" now support millisecond/microsecond unit and work with microsecond precision. [#52181](https://github.com/ClickHouse/ClickHouse/pull/52181) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create ZK ancestors optimistically. [#52195](https://github.com/ClickHouse/ClickHouse/pull/52195) ([Raúl Marín](https://github.com/Algunenano)). +* Fix [#50582](https://github.com/ClickHouse/ClickHouse/issues/50582). Avoid the `Not found column ... in block` error in some cases of reading in-order and constants. [#52259](https://github.com/ClickHouse/ClickHouse/pull/52259) ([Chen768959](https://github.com/Chen768959)). +* Check whether S2 geo primitives are invalid as early as possible on ClickHouse side. This closes: [#27090](https://github.com/ClickHouse/ClickHouse/issues/27090). [#52260](https://github.com/ClickHouse/ClickHouse/pull/52260) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now unquoted utf-8 strings are supported in DDL for MaterializedMySQL. [#52318](https://github.com/ClickHouse/ClickHouse/pull/52318) ([Val Doroshchuk](https://github.com/valbok)). +* Add back missing projection QueryAccessInfo when `query_plan_optimize_projection = 1`. This fixes [#50183](https://github.com/ClickHouse/ClickHouse/issues/50183) . This fixes [#50093](https://github.com/ClickHouse/ClickHouse/issues/50093) . [#52327](https://github.com/ClickHouse/ClickHouse/pull/52327) ([Amos Bird](https://github.com/amosbird)). +* Add new setting `disable_url_encoding` that allows to disable decoding/encoding path in uri in URL engine. [#52337](https://github.com/ClickHouse/ClickHouse/pull/52337) ([Kruglov Pavel](https://github.com/Avogar)). +* When `ZooKeeperRetriesControl` rethrows an error, it's more useful to see its original stack trace, not the one from `ZooKeeperRetriesControl` itself. [#52347](https://github.com/ClickHouse/ClickHouse/pull/52347) ([Vitaly Baranov](https://github.com/vitlibar)). +* Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). +* Wait for zero copy replication lock even if some disks don't support it. [#52376](https://github.com/ClickHouse/ClickHouse/pull/52376) ([Raúl Marín](https://github.com/Algunenano)). +* Now it's possible to specify min (`memory_profiler_sample_min_allocation_size`) and max (`memory_profiler_sample_max_allocation_size`) size for allocations to be tracked with sampling memory profiler. [#52419](https://github.com/ClickHouse/ClickHouse/pull/52419) ([alesapin](https://github.com/alesapin)). +* The `session_timezone` setting is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). +* Added field `refcount` to `system.remote_data_paths` table. [#52518](https://github.com/ClickHouse/ClickHouse/pull/52518) ([Anton Popov](https://github.com/CurtizJ)). +* New setting `merge_tree_determine_task_size_by_prewhere_columns` added. If set to `true` only sizes of the columns from `PREWHERE` section will be considered to determine reading task size. Otherwise all the columns from query are considered. [#52606](https://github.com/ClickHouse/ClickHouse/pull/52606) ([Nikita Taranov](https://github.com/nickitat)). + +#### Build/Testing/Packaging Improvement +* Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed CRC32(WeakHash32) issue for s390x. [#50365](https://github.com/ClickHouse/ClickHouse/pull/50365) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Add integration test check with the enabled analyzer. [#50926](https://github.com/ClickHouse/ClickHouse/pull/50926) ([Dmitry Novik](https://github.com/novikd)). +* Update cargo dependencies. [#51721](https://github.com/ClickHouse/ClickHouse/pull/51721) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed several issues found by OSS-Fuzz. [#51736](https://github.com/ClickHouse/ClickHouse/pull/51736) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* There were a couple of failures because of (?) S3 availability. The sccache has a feature of failing over to local compilation. [#51893](https://github.com/ClickHouse/ClickHouse/pull/51893) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* 02242_delete_user_race and 02243_drop_user_grant_race tests have been corrected. [#51923](https://github.com/ClickHouse/ClickHouse/pull/51923) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Make the function `CHColumnToArrowColumn::fillArrowArrayWithArrayColumnData` to work with nullable arrays, which are not possible in ClickHouse, but needed for Gluten. [#52112](https://github.com/ClickHouse/ClickHouse/pull/52112) ([李扬](https://github.com/taiyang-li)). +* We've updated the CCTZ library to master, but there are no user-visible changes. [#52124](https://github.com/ClickHouse/ClickHouse/pull/52124) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `system.licenses` table now includes the hard-forked library Poco. This closes [#52066](https://github.com/ClickHouse/ClickHouse/issues/52066). [#52127](https://github.com/ClickHouse/ClickHouse/pull/52127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Follow up [#50926](https://github.com/ClickHouse/ClickHouse/issues/50926). Add integration tests check with enabled analyzer to master. [#52210](https://github.com/ClickHouse/ClickHouse/pull/52210) ([Dmitry Novik](https://github.com/novikd)). +* Reproducible builds for Rust. [#52395](https://github.com/ClickHouse/ClickHouse/pull/52395) ([Azat Khuzhin](https://github.com/azat)). +* Improve the startup time of `clickhouse-client` and `clickhouse-local` in debug and sanitizer builds. This closes [#52228](https://github.com/ClickHouse/ClickHouse/issues/52228). [#52489](https://github.com/ClickHouse/ClickHouse/pull/52489) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that there are no cases of bad punctuation: whitespace before a comma like `Hello ,world` instead of `Hello, world`. [#52549](https://github.com/ClickHouse/ClickHouse/pull/52549) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix materialised pg syncTables [#49698](https://github.com/ClickHouse/ClickHouse/pull/49698) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix projection with optimize_aggregators_of_group_by_keys [#49709](https://github.com/ClickHouse/ClickHouse/pull/49709) ([Amos Bird](https://github.com/amosbird)). +* Fix optimize_skip_unused_shards with JOINs [#51037](https://github.com/ClickHouse/ClickHouse/pull/51037) ([Azat Khuzhin](https://github.com/azat)). +* Fix formatDateTime() with fractional negative datetime64 [#51290](https://github.com/ClickHouse/ClickHouse/pull/51290) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Functions `hasToken*` were totally wrong. Add a test for [#43358](https://github.com/ClickHouse/ClickHouse/issues/43358) [#51378](https://github.com/ClickHouse/ClickHouse/pull/51378) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix optimization to move functions before sorting. [#51481](https://github.com/ClickHouse/ClickHouse/pull/51481) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix Block structure mismatch in Pipe::unitePipes for FINAL [#51492](https://github.com/ClickHouse/ClickHouse/pull/51492) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SIGSEGV for clusters with zero weight across all shards (fixes INSERT INTO FUNCTION clusterAllReplicas()) [#51545](https://github.com/ClickHouse/ClickHouse/pull/51545) ([Azat Khuzhin](https://github.com/azat)). +* Fix timeout for hedged requests [#51582](https://github.com/ClickHouse/ClickHouse/pull/51582) ([Azat Khuzhin](https://github.com/azat)). +* Fix logical error in ANTI join with NULL [#51601](https://github.com/ClickHouse/ClickHouse/pull/51601) ([vdimir](https://github.com/vdimir)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Do not apply PredicateExpressionsOptimizer for ASOF/ANTI join [#51633](https://github.com/ClickHouse/ClickHouse/pull/51633) ([vdimir](https://github.com/vdimir)). +* Fix async insert with deduplication for ReplicatedMergeTree using merging algorithms [#51676](https://github.com/ClickHouse/ClickHouse/pull/51676) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Fix segfault when create invalid EmbeddedRocksdb table [#51847](https://github.com/ClickHouse/ClickHouse/pull/51847) ([Duc Canh Le](https://github.com/canhld94)). +* Fix inserts into MongoDB tables [#51876](https://github.com/ClickHouse/ClickHouse/pull/51876) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix deadlock on DatabaseCatalog shutdown [#51908](https://github.com/ClickHouse/ClickHouse/pull/51908) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix error in subquery operators [#51922](https://github.com/ClickHouse/ClickHouse/pull/51922) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix async connect to hosts with multiple ips [#51934](https://github.com/ClickHouse/ClickHouse/pull/51934) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not remove inputs after ActionsDAG::merge [#51947](https://github.com/ClickHouse/ClickHouse/pull/51947) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` [#51954](https://github.com/ClickHouse/ClickHouse/pull/51954) ([vdimir](https://github.com/vdimir)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix for toDateTime64() for dates after 2283-12-31 [#52130](https://github.com/ClickHouse/ClickHouse/pull/52130) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix ORDER BY tuple of WINDOW functions [#52145](https://github.com/ClickHouse/ClickHouse/pull/52145) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis when aggregation expression contains monotonic functions [#52151](https://github.com/ClickHouse/ClickHouse/pull/52151) ([Amos Bird](https://github.com/amosbird)). +* Fix error in `groupArrayMoving` functions [#52161](https://github.com/ClickHouse/ClickHouse/pull/52161) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable direct join for range dictionary [#52187](https://github.com/ClickHouse/ClickHouse/pull/52187) ([Duc Canh Le](https://github.com/canhld94)). +* Fix sticky mutations test (and extremely rare race condition) [#52197](https://github.com/ClickHouse/ClickHouse/pull/52197) ([alesapin](https://github.com/alesapin)). +* Fix race in Web disk [#52211](https://github.com/ClickHouse/ClickHouse/pull/52211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in Connection::setAsyncCallback on unknown packet from server [#52219](https://github.com/ClickHouse/ClickHouse/pull/52219) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix temp data deletion on startup, add test [#52275](https://github.com/ClickHouse/ClickHouse/pull/52275) ([vdimir](https://github.com/vdimir)). +* Don't use minmax_count projections when counting nullable columns [#52297](https://github.com/ClickHouse/ClickHouse/pull/52297) ([Amos Bird](https://github.com/amosbird)). +* MergeTree/ReplicatedMergeTree should use server timezone for log entries [#52325](https://github.com/ClickHouse/ClickHouse/pull/52325) ([Azat Khuzhin](https://github.com/azat)). +* Fix parameterized view with cte and multiple usage [#52328](https://github.com/ClickHouse/ClickHouse/pull/52328) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Disable expression templates for time intervals [#52335](https://github.com/ClickHouse/ClickHouse/pull/52335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `apply_snapshot` in Keeper [#52358](https://github.com/ClickHouse/ClickHouse/pull/52358) ([Antonio Andelic](https://github.com/antonio2368)). +* Update build-osx.md [#52377](https://github.com/ClickHouse/ClickHouse/pull/52377) ([AlexBykovski](https://github.com/AlexBykovski)). +* Fix `countSubstrings()` hang with empty needle and a column haystack [#52409](https://github.com/ClickHouse/ClickHouse/pull/52409) ([Sergei Trifonov](https://github.com/serxa)). +* Fix normal projection with merge table [#52432](https://github.com/ClickHouse/ClickHouse/pull/52432) ([Amos Bird](https://github.com/amosbird)). +* Fix possible double-free in Aggregator [#52439](https://github.com/ClickHouse/ClickHouse/pull/52439) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed inserting into Buffer engine [#52440](https://github.com/ClickHouse/ClickHouse/pull/52440) ([Vasily Nemkov](https://github.com/Enmk)). +* The implementation of AnyHash was non-conformant. [#52448](https://github.com/ClickHouse/ClickHouse/pull/52448) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check recursion depth in OptimizedRegularExpression [#52451](https://github.com/ClickHouse/ClickHouse/pull/52451) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() [#52490](https://github.com/ClickHouse/ClickHouse/pull/52490) ([Azat Khuzhin](https://github.com/azat)). +* Fix abort in function `transform` [#52513](https://github.com/ClickHouse/ClickHouse/pull/52513) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix lightweight delete after drop of projection [#52517](https://github.com/ClickHouse/ClickHouse/pull/52517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error "Cannot drain connections: cancel first" [#52585](https://github.com/ClickHouse/ClickHouse/pull/52585) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Add documentation for building in docker"'. [#51773](https://github.com/ClickHouse/ClickHouse/pull/51773) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Fix build"'. [#51911](https://github.com/ClickHouse/ClickHouse/pull/51911) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Millisecond and microsecond support in date_diff / age functions"'. [#52129](https://github.com/ClickHouse/ClickHouse/pull/52129) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Re-add SipHash keyed functions"'. [#52466](https://github.com/ClickHouse/ClickHouse/pull/52466) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Add an ability to specify allocations size for sampling memory profiler"'. [#52496](https://github.com/ClickHouse/ClickHouse/pull/52496) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Rewrite uniq to count"'. [#52576](https://github.com/ClickHouse/ClickHouse/pull/52576) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Remove duplicate_order_by_and_distinct optimization [#47135](https://github.com/ClickHouse/ClickHouse/pull/47135) ([Igor Nikonov](https://github.com/devcrafter)). +* Update sort desc in ReadFromMergeTree after applying PREWHERE info [#48669](https://github.com/ClickHouse/ClickHouse/pull/48669) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix `BindException: Address already in use` in HDFS integration tests [#49428](https://github.com/ClickHouse/ClickHouse/pull/49428) ([Nikita Taranov](https://github.com/nickitat)). +* Force libunwind usage (removes gcc_eh support) [#49438](https://github.com/ClickHouse/ClickHouse/pull/49438) ([Azat Khuzhin](https://github.com/azat)). +* Cleanup `storage_conf.xml` [#49557](https://github.com/ClickHouse/ClickHouse/pull/49557) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky tests caused by OPTIMIZE FINAL failing memory budget check [#49764](https://github.com/ClickHouse/ClickHouse/pull/49764) ([Michael Kolupaev](https://github.com/al13n321)). +* Remove unstable queries from performance/join_set_filter [#50235](https://github.com/ClickHouse/ClickHouse/pull/50235) ([vdimir](https://github.com/vdimir)). +* More accurate DNS resolve for the keeper connection [#50738](https://github.com/ClickHouse/ClickHouse/pull/50738) ([pufit](https://github.com/pufit)). +* Try to fix some trash in Disks and part moves [#51135](https://github.com/ClickHouse/ClickHouse/pull/51135) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add jemalloc support fro s390x [#51186](https://github.com/ClickHouse/ClickHouse/pull/51186) ([Boris Kuschel](https://github.com/bkuschel)). +* Resubmit [#48821](https://github.com/ClickHouse/ClickHouse/issues/48821) [#51208](https://github.com/ClickHouse/ClickHouse/pull/51208) ([Kseniia Sumarokova](https://github.com/kssenii)). +* test for [#36894](https://github.com/ClickHouse/ClickHouse/issues/36894) [#51274](https://github.com/ClickHouse/ClickHouse/pull/51274) ([Denny Crane](https://github.com/den-crane)). +* external_aggregation_fix for big endian machines [#51280](https://github.com/ClickHouse/ClickHouse/pull/51280) ([Sanjam Panda](https://github.com/saitama951)). +* Fix: Invalid number of rows in Chunk column Object [#51296](https://github.com/ClickHouse/ClickHouse/pull/51296) ([Igor Nikonov](https://github.com/devcrafter)). +* Add a test for [#44816](https://github.com/ClickHouse/ClickHouse/issues/44816) [#51305](https://github.com/ClickHouse/ClickHouse/pull/51305) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for `calculate_text_stack_trace` setting [#51311](https://github.com/ClickHouse/ClickHouse/pull/51311) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* decrease log level, make logs shorter [#51320](https://github.com/ClickHouse/ClickHouse/pull/51320) ([Sema Checherinda](https://github.com/CheSema)). +* Collect stack traces from job's scheduling and print along with exception's stack trace. [#51349](https://github.com/ClickHouse/ClickHouse/pull/51349) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add a test for [#42691](https://github.com/ClickHouse/ClickHouse/issues/42691) [#51352](https://github.com/ClickHouse/ClickHouse/pull/51352) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#32474](https://github.com/ClickHouse/ClickHouse/issues/32474) [#51354](https://github.com/ClickHouse/ClickHouse/pull/51354) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#41727](https://github.com/ClickHouse/ClickHouse/issues/41727) [#51355](https://github.com/ClickHouse/ClickHouse/pull/51355) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#35801](https://github.com/ClickHouse/ClickHouse/issues/35801) [#51356](https://github.com/ClickHouse/ClickHouse/pull/51356) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#34626](https://github.com/ClickHouse/ClickHouse/issues/34626) [#51357](https://github.com/ClickHouse/ClickHouse/pull/51357) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Initialize text_log earlier to capture table startup messages [#51360](https://github.com/ClickHouse/ClickHouse/pull/51360) ([Azat Khuzhin](https://github.com/azat)). +* Use separate default settings for clickhouse-local [#51363](https://github.com/ClickHouse/ClickHouse/pull/51363) ([Azat Khuzhin](https://github.com/azat)). +* Attempt to remove wrong code (catch/throw in Functions) [#51367](https://github.com/ClickHouse/ClickHouse/pull/51367) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove suspicious code [#51383](https://github.com/ClickHouse/ClickHouse/pull/51383) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable hedged requests under TSan [#51392](https://github.com/ClickHouse/ClickHouse/pull/51392) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* no finalize in d-tor WriteBufferFromOStream [#51404](https://github.com/ClickHouse/ClickHouse/pull/51404) ([Sema Checherinda](https://github.com/CheSema)). +* Better diagnostics for 01193_metadata_loading [#51414](https://github.com/ClickHouse/ClickHouse/pull/51414) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix attaching gdb in stress tests [#51445](https://github.com/ClickHouse/ClickHouse/pull/51445) ([Kruglov Pavel](https://github.com/Avogar)). +* Merging [#36384](https://github.com/ClickHouse/ClickHouse/issues/36384) [#51458](https://github.com/ClickHouse/ClickHouse/pull/51458) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible race on shutdown wait [#51497](https://github.com/ClickHouse/ClickHouse/pull/51497) ([Sergei Trifonov](https://github.com/serxa)). +* Fix `test_alter_moving_garbage`: lock between getActiveContainingPart and swapActivePart in parts mover [#51498](https://github.com/ClickHouse/ClickHouse/pull/51498) ([vdimir](https://github.com/vdimir)). +* Fix a logical error on mutation [#51502](https://github.com/ClickHouse/ClickHouse/pull/51502) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix running integration tests with spaces in it's names [#51514](https://github.com/ClickHouse/ClickHouse/pull/51514) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky test 00417_kill_query [#51522](https://github.com/ClickHouse/ClickHouse/pull/51522) ([Nikolay Degterinsky](https://github.com/evillique)). +* fs cache: add some checks [#51536](https://github.com/ClickHouse/ClickHouse/pull/51536) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't run 02782_uniq_exact_parallel_merging_bug in parallel with other tests [#51549](https://github.com/ClickHouse/ClickHouse/pull/51549) ([Nikita Taranov](https://github.com/nickitat)). +* 00900_orc_load: lift kill timeout [#51559](https://github.com/ClickHouse/ClickHouse/pull/51559) ([Robert Schulze](https://github.com/rschu1ze)). +* Add retries to 00416_pocopatch_progress_in_http_headers [#51575](https://github.com/ClickHouse/ClickHouse/pull/51575) ([Nikolay Degterinsky](https://github.com/evillique)). +* Remove the usage of Analyzer setting in the client [#51578](https://github.com/ClickHouse/ClickHouse/pull/51578) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix merge_selecting_task scheduling [#51591](https://github.com/ClickHouse/ClickHouse/pull/51591) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add hex functions for cityhash [#51595](https://github.com/ClickHouse/ClickHouse/pull/51595) ([Vitaly Baranov](https://github.com/vitlibar)). +* Remove `unset CLICKHOUSE_LOG_COMMENT` from tests [#51623](https://github.com/ClickHouse/ClickHouse/pull/51623) ([Nikita Taranov](https://github.com/nickitat)). +* Implement endianness-independent serialization [#51637](https://github.com/ClickHouse/ClickHouse/pull/51637) ([ltrk2](https://github.com/ltrk2)). +* Ignore APPEND and TRUNCATE modifiers if file does not exist. [#51640](https://github.com/ClickHouse/ClickHouse/pull/51640) ([alekar](https://github.com/alekar)). +* Try to fix flaky 02210_processors_profile_log [#51641](https://github.com/ClickHouse/ClickHouse/pull/51641) ([Igor Nikonov](https://github.com/devcrafter)). +* Make common macros extendable [#51646](https://github.com/ClickHouse/ClickHouse/pull/51646) ([Amos Bird](https://github.com/amosbird)). +* Correct an exception message in src/Functions/nested.cpp [#51651](https://github.com/ClickHouse/ClickHouse/pull/51651) ([Alex Cheng](https://github.com/Alex-Cheng)). +* tests: fix 02050_client_profile_events flakiness [#51653](https://github.com/ClickHouse/ClickHouse/pull/51653) ([Azat Khuzhin](https://github.com/azat)). +* Minor follow-up to re2 update to 2023-06-02 ([#50949](https://github.com/ClickHouse/ClickHouse/issues/50949)) [#51655](https://github.com/ClickHouse/ClickHouse/pull/51655) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix 02116_tuple_element with Analyzer [#51669](https://github.com/ClickHouse/ClickHouse/pull/51669) ([Robert Schulze](https://github.com/rschu1ze)). +* Update timeouts in tests for transactions [#51683](https://github.com/ClickHouse/ClickHouse/pull/51683) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove unused code [#51684](https://github.com/ClickHouse/ClickHouse/pull/51684) ([Sergei Trifonov](https://github.com/serxa)). +* Remove `mmap/mremap/munmap` from Allocator.h [#51686](https://github.com/ClickHouse/ClickHouse/pull/51686) ([alesapin](https://github.com/alesapin)). +* SonarCloud: Add C++23 Experimental Flag [#51687](https://github.com/ClickHouse/ClickHouse/pull/51687) ([Julio Jimenez](https://github.com/juliojimenez)). +* Wait with retries when attaching GDB in tests [#51688](https://github.com/ClickHouse/ClickHouse/pull/51688) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v23.6.1.1524-stable [#51691](https://github.com/ClickHouse/ClickHouse/pull/51691) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* fix write to finalized buffer [#51696](https://github.com/ClickHouse/ClickHouse/pull/51696) ([Sema Checherinda](https://github.com/CheSema)). +* do not log exception aborted for pending mutate/merge entries when shutdown [#51697](https://github.com/ClickHouse/ClickHouse/pull/51697) ([Sema Checherinda](https://github.com/CheSema)). +* Fix race in ContextAccess [#51704](https://github.com/ClickHouse/ClickHouse/pull/51704) ([Vitaly Baranov](https://github.com/vitlibar)). +* Make test scripts backwards compatible [#51707](https://github.com/ClickHouse/ClickHouse/pull/51707) ([Antonio Andelic](https://github.com/antonio2368)). +* test for full join and null predicate [#51709](https://github.com/ClickHouse/ClickHouse/pull/51709) ([Denny Crane](https://github.com/den-crane)). +* A cmake warning on job limits underutilizing CPU [#51710](https://github.com/ClickHouse/ClickHouse/pull/51710) ([velavokr](https://github.com/velavokr)). +* Fix SQLLogic docker images [#51719](https://github.com/ClickHouse/ClickHouse/pull/51719) ([Antonio Andelic](https://github.com/antonio2368)). +* Added ASK_PASSWORD client constant instead of hardcoded '\n' [#51723](https://github.com/ClickHouse/ClickHouse/pull/51723) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Update README.md [#51726](https://github.com/ClickHouse/ClickHouse/pull/51726) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fix source image for sqllogic [#51728](https://github.com/ClickHouse/ClickHouse/pull/51728) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove MemoryPool from Poco because it's useless [#51732](https://github.com/ClickHouse/ClickHouse/pull/51732) ([alesapin](https://github.com/alesapin)). +* Fix: logical error in grace hash join [#51737](https://github.com/ClickHouse/ClickHouse/pull/51737) ([Igor Nikonov](https://github.com/devcrafter)). +* Update 01320_create_sync_race_condition_zookeeper.sh [#51742](https://github.com/ClickHouse/ClickHouse/pull/51742) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Pin for docker-ce [#51743](https://github.com/ClickHouse/ClickHouse/pull/51743) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Revert "Fix: Invalid number of rows in Chunk column Object" [#51750](https://github.com/ClickHouse/ClickHouse/pull/51750) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add SonarCloud to README [#51751](https://github.com/ClickHouse/ClickHouse/pull/51751) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix test `02789_object_type_invalid_num_of_rows` [#51754](https://github.com/ClickHouse/ClickHouse/pull/51754) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix (benign) data race in `transform` [#51755](https://github.com/ClickHouse/ClickHouse/pull/51755) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky KeeperMap test [#51764](https://github.com/ClickHouse/ClickHouse/pull/51764) ([Antonio Andelic](https://github.com/antonio2368)). +* Version mypy=1.4.1 falsly reports unused ignore comment [#51769](https://github.com/ClickHouse/ClickHouse/pull/51769) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid keeping lock Context::getLock() while calculating access rights [#51772](https://github.com/ClickHouse/ClickHouse/pull/51772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Making stateless tests with timeout less flaky [#51774](https://github.com/ClickHouse/ClickHouse/pull/51774) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix after [#51000](https://github.com/ClickHouse/ClickHouse/issues/51000) [#51790](https://github.com/ClickHouse/ClickHouse/pull/51790) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add assert in ThreadStatus destructor for correct current_thread [#51800](https://github.com/ClickHouse/ClickHouse/pull/51800) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix broken parts handling in `ReplicatedMergeTree` [#51801](https://github.com/ClickHouse/ClickHouse/pull/51801) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix tsan signal-unsafe call [#51802](https://github.com/ClickHouse/ClickHouse/pull/51802) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix for parallel replicas not completely disabled by granule count threshold [#51805](https://github.com/ClickHouse/ClickHouse/pull/51805) ([Alexander Gololobov](https://github.com/davenger)). +* Make sure that we don't attempt to serialize/deserialize block with 0 columns and non-zero rows [#51807](https://github.com/ClickHouse/ClickHouse/pull/51807) ([Alexander Gololobov](https://github.com/davenger)). +* Fix rare bug in `DROP COLUMN` and enabled sparse columns [#51809](https://github.com/ClickHouse/ClickHouse/pull/51809) ([Anton Popov](https://github.com/CurtizJ)). +* Fix flaky `test_multiple_disks` [#51821](https://github.com/ClickHouse/ClickHouse/pull/51821) ([Antonio Andelic](https://github.com/antonio2368)). +* Follow up to [#51547](https://github.com/ClickHouse/ClickHouse/issues/51547) [#51822](https://github.com/ClickHouse/ClickHouse/pull/51822) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Correctly grep archives in stress tests [#51824](https://github.com/ClickHouse/ClickHouse/pull/51824) ([Antonio Andelic](https://github.com/antonio2368)). +* Update analyzer_tech_debt.txt [#51836](https://github.com/ClickHouse/ClickHouse/pull/51836) ([Alexander Tokmakov](https://github.com/tavplubix)). +* remove unused code [#51837](https://github.com/ClickHouse/ClickHouse/pull/51837) ([flynn](https://github.com/ucasfl)). +* Fix disk config for upgrade tests [#51839](https://github.com/ClickHouse/ClickHouse/pull/51839) ([Antonio Andelic](https://github.com/antonio2368)). +* Remove Coverity from workflows, but leave in the code [#51842](https://github.com/ClickHouse/ClickHouse/pull/51842) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Many fixes [3] [#51848](https://github.com/ClickHouse/ClickHouse/pull/51848) ([Ilya Yatsishin](https://github.com/qoega)). +* Change misleading name in joins: addJoinedBlock -> addBlockToJoin [#51852](https://github.com/ClickHouse/ClickHouse/pull/51852) ([Igor Nikonov](https://github.com/devcrafter)). +* fix: correct exception messages on policies comparison [#51854](https://github.com/ClickHouse/ClickHouse/pull/51854) ([Feng Kaiyu](https://github.com/fky2015)). +* Update 02439_merge_selecting_partitions.sql [#51862](https://github.com/ClickHouse/ClickHouse/pull/51862) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove useless packages [#51863](https://github.com/ClickHouse/ClickHouse/pull/51863) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove useless logs [#51865](https://github.com/ClickHouse/ClickHouse/pull/51865) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect log level = warning [#51867](https://github.com/ClickHouse/ClickHouse/pull/51867) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_replicated_table_attach [#51868](https://github.com/ClickHouse/ClickHouse/pull/51868) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better usability of a test [#51869](https://github.com/ClickHouse/ClickHouse/pull/51869) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove useless code [#51873](https://github.com/ClickHouse/ClickHouse/pull/51873) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Another fix upgrade check script [#51878](https://github.com/ClickHouse/ClickHouse/pull/51878) ([Antonio Andelic](https://github.com/antonio2368)). +* Sqlloogic improvements [#51883](https://github.com/ClickHouse/ClickHouse/pull/51883) ([Ilya Yatsishin](https://github.com/qoega)). +* Disable ThinLTO on non-Linux [#51897](https://github.com/ClickHouse/ClickHouse/pull/51897) ([Robert Schulze](https://github.com/rschu1ze)). +* Pin rust nightly (to make it stable) [#51903](https://github.com/ClickHouse/ClickHouse/pull/51903) ([Azat Khuzhin](https://github.com/azat)). +* Fix build [#51909](https://github.com/ClickHouse/ClickHouse/pull/51909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix build [#51910](https://github.com/ClickHouse/ClickHouse/pull/51910) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test `00175_partition_by_ignore` and move it to correct location [#51913](https://github.com/ClickHouse/ClickHouse/pull/51913) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test 02360_send_logs_level_colors: avoid usage of `file` tool [#51914](https://github.com/ClickHouse/ClickHouse/pull/51914) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Maybe better tests [#51916](https://github.com/ClickHouse/ClickHouse/pull/51916) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert system drop filesystem cache by key [#51917](https://github.com/ClickHouse/ClickHouse/pull/51917) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test `detach_attach_partition_race` [#51920](https://github.com/ClickHouse/ClickHouse/pull/51920) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Another fix for `02481_async_insert_race_long` [#51925](https://github.com/ClickHouse/ClickHouse/pull/51925) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix segfault caused by `ThreadStatus` [#51931](https://github.com/ClickHouse/ClickHouse/pull/51931) ([Antonio Andelic](https://github.com/antonio2368)). +* Print short fault info only from safe fields [#51932](https://github.com/ClickHouse/ClickHouse/pull/51932) ([Alexander Gololobov](https://github.com/davenger)). +* Fix typo in integration tests [#51944](https://github.com/ClickHouse/ClickHouse/pull/51944) ([Ilya Yatsishin](https://github.com/qoega)). +* Better logs on shutdown [#51951](https://github.com/ClickHouse/ClickHouse/pull/51951) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Filter databases list before querying potentially slow fields [#51955](https://github.com/ClickHouse/ClickHouse/pull/51955) ([Alexander Gololobov](https://github.com/davenger)). +* Fix some issues with transactions [#51959](https://github.com/ClickHouse/ClickHouse/pull/51959) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix unrelated messages from LSan in clickhouse-client [#51966](https://github.com/ClickHouse/ClickHouse/pull/51966) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow OOM in AST Fuzzer with Sanitizers [#51967](https://github.com/ClickHouse/ClickHouse/pull/51967) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable one test under Analyzer [#51968](https://github.com/ClickHouse/ClickHouse/pull/51968) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix Docker [#51969](https://github.com/ClickHouse/ClickHouse/pull/51969) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `01825_type_json_from_map` [#51970](https://github.com/ClickHouse/ClickHouse/pull/51970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `02354_distributed_with_external_aggregation_memory_usage` [#51971](https://github.com/ClickHouse/ClickHouse/pull/51971) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix disaster in integration tests, part 2 [#51973](https://github.com/ClickHouse/ClickHouse/pull/51973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [RFC] Cleanup remote_servers in dist config.xml [#51985](https://github.com/ClickHouse/ClickHouse/pull/51985) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.6.2.18-stable [#51986](https://github.com/ClickHouse/ClickHouse/pull/51986) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.8.20.11-lts [#51987](https://github.com/ClickHouse/ClickHouse/pull/51987) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix performance test for regexp cache [#51988](https://github.com/ClickHouse/ClickHouse/pull/51988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Move a test to the right place [#51989](https://github.com/ClickHouse/ClickHouse/pull/51989) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a check to validate that the stateful tests are stateful [#51990](https://github.com/ClickHouse/ClickHouse/pull/51990) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that functional tests cleanup their tables [#51991](https://github.com/ClickHouse/ClickHouse/pull/51991) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_extreme_deduplication [#51992](https://github.com/ClickHouse/ClickHouse/pull/51992) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Cleanup SymbolIndex after reload got removed [#51993](https://github.com/ClickHouse/ClickHouse/pull/51993) ([Azat Khuzhin](https://github.com/azat)). +* Update CompletedPipelineExecutor exception log name [#52028](https://github.com/ClickHouse/ClickHouse/pull/52028) ([xiao](https://github.com/nicelulu)). +* Fix `00502_custom_partitioning_replicated_zookeeper_long` [#52032](https://github.com/ClickHouse/ClickHouse/pull/52032) ([Antonio Andelic](https://github.com/antonio2368)). +* Prohibit send_metadata for s3_plain disks [#52038](https://github.com/ClickHouse/ClickHouse/pull/52038) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.4.6.25-stable [#52061](https://github.com/ClickHouse/ClickHouse/pull/52061) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Preparations for Trivial Support For Resharding (part1) [#52068](https://github.com/ClickHouse/ClickHouse/pull/52068) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.3.8.21-lts [#52077](https://github.com/ClickHouse/ClickHouse/pull/52077) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix flakiness of test_keeper_s3_snapshot flakiness [#52083](https://github.com/ClickHouse/ClickHouse/pull/52083) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_extreme_deduplication flakiness [#52085](https://github.com/ClickHouse/ClickHouse/pull/52085) ([Azat Khuzhin](https://github.com/azat)). +* Small docs update for toYearWeek() function [#52090](https://github.com/ClickHouse/ClickHouse/pull/52090) ([Andrey Zvonov](https://github.com/zvonand)). +* Small docs update for DateTime, DateTime64 [#52094](https://github.com/ClickHouse/ClickHouse/pull/52094) ([Andrey Zvonov](https://github.com/zvonand)). +* Add missing --force for docker network prune (otherwise it is noop on CI) [#52095](https://github.com/ClickHouse/ClickHouse/pull/52095) ([Azat Khuzhin](https://github.com/azat)). +* tests: drop existing view in test_materialized_mysql_database [#52103](https://github.com/ClickHouse/ClickHouse/pull/52103) ([Azat Khuzhin](https://github.com/azat)). +* Update README.md [#52115](https://github.com/ClickHouse/ClickHouse/pull/52115) ([Tyler Hannan](https://github.com/tylerhannan)). +* Print Zxid in keeper stat command in hex (so as ZooKeeper) [#52122](https://github.com/ClickHouse/ClickHouse/pull/52122) ([Azat Khuzhin](https://github.com/azat)). +* Skip protection from double decompression if inode from maps cannot be obtained [#52138](https://github.com/ClickHouse/ClickHouse/pull/52138) ([Azat Khuzhin](https://github.com/azat)). +* There is no point in detecting flaky tests [#52142](https://github.com/ClickHouse/ClickHouse/pull/52142) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove default argument value [#52143](https://github.com/ClickHouse/ClickHouse/pull/52143) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the "kill_mutation" test [#52144](https://github.com/ClickHouse/ClickHouse/pull/52144) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix ORDER BY tuple of WINDOW functions (and slightly more changes) [#52146](https://github.com/ClickHouse/ClickHouse/pull/52146) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible EADDRINUSE ("Address already in use") in integration tests [#52148](https://github.com/ClickHouse/ClickHouse/pull/52148) ([Azat Khuzhin](https://github.com/azat)). +* Fix test 02497_storage_file_reader_selection [#52154](https://github.com/ClickHouse/ClickHouse/pull/52154) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix unexpected AST Set [#52158](https://github.com/ClickHouse/ClickHouse/pull/52158) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix crash in comparison functions due to incorrect query analysis [#52172](https://github.com/ClickHouse/ClickHouse/pull/52172) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix slow test `02317_distinct_in_order_optimization` [#52173](https://github.com/ClickHouse/ClickHouse/pull/52173) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add comments for https://github.com/ClickHouse/ClickHouse/pull/52112 [#52175](https://github.com/ClickHouse/ClickHouse/pull/52175) ([李扬](https://github.com/taiyang-li)). +* Randomize timezone in tests across non-deterministic around 1970 and default [#52184](https://github.com/ClickHouse/ClickHouse/pull/52184) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_multiple_disks/test.py::test_start_stop_moves` [#52189](https://github.com/ClickHouse/ClickHouse/pull/52189) ([Antonio Andelic](https://github.com/antonio2368)). +* CMake: Simplify job limiting [#52196](https://github.com/ClickHouse/ClickHouse/pull/52196) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix self extracting binaries under qemu linux-user (qemu-$ARCH-static) [#52198](https://github.com/ClickHouse/ClickHouse/pull/52198) ([Azat Khuzhin](https://github.com/azat)). +* Fix `Integration tests flaky check (asan)` [#52201](https://github.com/ClickHouse/ClickHouse/pull/52201) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky test test_lost_part [#52202](https://github.com/ClickHouse/ClickHouse/pull/52202) ([alesapin](https://github.com/alesapin)). +* MaterializedMySQL: Replace to_string by magic_enum::enum_name [#52204](https://github.com/ClickHouse/ClickHouse/pull/52204) ([Val Doroshchuk](https://github.com/valbok)). +* MaterializedMySQL: Add tests to parse db and table names from DDL [#52208](https://github.com/ClickHouse/ClickHouse/pull/52208) ([Val Doroshchuk](https://github.com/valbok)). +* Revert "Fixed several issues found by OSS-Fuzz" [#52216](https://github.com/ClickHouse/ClickHouse/pull/52216) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Use one copy replication more agressively [#52218](https://github.com/ClickHouse/ClickHouse/pull/52218) ([alesapin](https://github.com/alesapin)). +* Fix flaky test `01076_parallel_alter_replicated_zookeeper` [#52221](https://github.com/ClickHouse/ClickHouse/pull/52221) ([alesapin](https://github.com/alesapin)). +* Fix 01889_key_condition_function_chains for analyzer. [#52223](https://github.com/ClickHouse/ClickHouse/pull/52223) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Inhibit settings randomization in the test `json_ghdata` [#52226](https://github.com/ClickHouse/ClickHouse/pull/52226) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Slightly better diagnostics in a test [#52227](https://github.com/ClickHouse/ClickHouse/pull/52227) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable no-upgrade-check for 02273_full_sort_join [#52235](https://github.com/ClickHouse/ClickHouse/pull/52235) ([vdimir](https://github.com/vdimir)). +* Fix network manager for integration tests [#52237](https://github.com/ClickHouse/ClickHouse/pull/52237) ([Azat Khuzhin](https://github.com/azat)). +* List replication queue only for current test database [#52238](https://github.com/ClickHouse/ClickHouse/pull/52238) ([Alexander Gololobov](https://github.com/davenger)). +* Attempt to fix assert in tsan with fibers [#52241](https://github.com/ClickHouse/ClickHouse/pull/52241) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix undefined behaviour in fuzzer [#52256](https://github.com/ClickHouse/ClickHouse/pull/52256) ([Antonio Andelic](https://github.com/antonio2368)). +* Follow-up to [#51959](https://github.com/ClickHouse/ClickHouse/issues/51959) [#52261](https://github.com/ClickHouse/ClickHouse/pull/52261) ([Alexander Tokmakov](https://github.com/tavplubix)). +* More fair queue for `drop table sync` [#52276](https://github.com/ClickHouse/ClickHouse/pull/52276) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `02497_trace_events_stress_long` [#52279](https://github.com/ClickHouse/ClickHouse/pull/52279) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix test `01111_create_drop_replicated_db_stress` [#52283](https://github.com/ClickHouse/ClickHouse/pull/52283) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix ugly code [#52284](https://github.com/ClickHouse/ClickHouse/pull/52284) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add missing replica syncs in test_backup_restore_on_cluster [#52306](https://github.com/ClickHouse/ClickHouse/pull/52306) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix test_replicated_database 'node doesn't exist' flakiness [#52307](https://github.com/ClickHouse/ClickHouse/pull/52307) ([Michael Kolupaev](https://github.com/al13n321)). +* Minor: Update description of events "QueryCacheHits/Misses" [#52309](https://github.com/ClickHouse/ClickHouse/pull/52309) ([Robert Schulze](https://github.com/rschu1ze)). +* Beautify pretty-printing of the query string in SYSTEM.QUERY_CACHE [#52312](https://github.com/ClickHouse/ClickHouse/pull/52312) ([Robert Schulze](https://github.com/rschu1ze)). +* Reduce dependencies for skim by avoid using default features [#52316](https://github.com/ClickHouse/ClickHouse/pull/52316) ([Azat Khuzhin](https://github.com/azat)). +* Fix 02725_memory-for-merges [#52317](https://github.com/ClickHouse/ClickHouse/pull/52317) ([alesapin](https://github.com/alesapin)). +* Skip unsupported disks in Keeper [#52321](https://github.com/ClickHouse/ClickHouse/pull/52321) ([Antonio Andelic](https://github.com/antonio2368)). +* Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed" [#52322](https://github.com/ClickHouse/ClickHouse/pull/52322) ([Kruglov Pavel](https://github.com/Avogar)). +* Resubmit [#51716](https://github.com/ClickHouse/ClickHouse/issues/51716) [#52323](https://github.com/ClickHouse/ClickHouse/pull/52323) ([Kruglov Pavel](https://github.com/Avogar)). +* Add logging about all found workflows for merge_pr.py [#52324](https://github.com/ClickHouse/ClickHouse/pull/52324) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Minor: Less awkward IAST::FormatSettings [#52332](https://github.com/ClickHouse/ClickHouse/pull/52332) ([Robert Schulze](https://github.com/rschu1ze)). +* Mark test 02125_many_mutations_2 as no-parallel to avoid flakiness [#52338](https://github.com/ClickHouse/ClickHouse/pull/52338) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix capabilities installed via systemd service (fixes netlink/IO priorities) [#52357](https://github.com/ClickHouse/ClickHouse/pull/52357) ([Azat Khuzhin](https://github.com/azat)). +* Update 01606_git_import.sh [#52360](https://github.com/ClickHouse/ClickHouse/pull/52360) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update ci-slack-bot.py [#52372](https://github.com/ClickHouse/ClickHouse/pull/52372) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `test_keeper_session` [#52373](https://github.com/ClickHouse/ClickHouse/pull/52373) ([Antonio Andelic](https://github.com/antonio2368)). +* Update ci-slack-bot.py [#52374](https://github.com/ClickHouse/ClickHouse/pull/52374) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable analyzer setting in backward_compatibility integration tests. [#52375](https://github.com/ClickHouse/ClickHouse/pull/52375) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* New metric - Filesystem cache size limit [#52378](https://github.com/ClickHouse/ClickHouse/pull/52378) ([Krzysztof Góralski](https://github.com/kgoralski)). +* Fix `test_replicated_merge_tree_encrypted_disk ` [#52379](https://github.com/ClickHouse/ClickHouse/pull/52379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix `02122_parallel_formatting_XML ` [#52380](https://github.com/ClickHouse/ClickHouse/pull/52380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Follow up to [#49698](https://github.com/ClickHouse/ClickHouse/issues/49698) [#52381](https://github.com/ClickHouse/ClickHouse/pull/52381) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Less replication errors [#52382](https://github.com/ClickHouse/ClickHouse/pull/52382) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Rename TaskStatsInfoGetter into NetlinkMetricsProvider [#52392](https://github.com/ClickHouse/ClickHouse/pull/52392) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_keeper_force_recovery` [#52408](https://github.com/ClickHouse/ClickHouse/pull/52408) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky gtest_lru_file_cache.cpp [#52418](https://github.com/ClickHouse/ClickHouse/pull/52418) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix: remove redundant distinct with views [#52438](https://github.com/ClickHouse/ClickHouse/pull/52438) ([Igor Nikonov](https://github.com/devcrafter)). +* Add 02815_range_dict_no_direct_join to analyzer_tech_debt.txt [#52464](https://github.com/ClickHouse/ClickHouse/pull/52464) ([vdimir](https://github.com/vdimir)). +* do not throw exception in OptimizedRegularExpressionImpl::analyze [#52467](https://github.com/ClickHouse/ClickHouse/pull/52467) ([Han Fei](https://github.com/hanfei1991)). +* Remove skip_startup_tables from IDatabase::loadStoredObjects() [#52491](https://github.com/ClickHouse/ClickHouse/pull/52491) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_insert_same_partition_and_merge by increasing wait time [#52497](https://github.com/ClickHouse/ClickHouse/pull/52497) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Try to fix asan wanring in HashJoin [#52499](https://github.com/ClickHouse/ClickHouse/pull/52499) ([Igor Nikonov](https://github.com/devcrafter)). +* Replace with three way comparison [#52509](https://github.com/ClickHouse/ClickHouse/pull/52509) ([flynn](https://github.com/ucasfl)). +* Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop [#52514](https://github.com/ClickHouse/ClickHouse/pull/52514) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_throttling` [#52515](https://github.com/ClickHouse/ClickHouse/pull/52515) ([Antonio Andelic](https://github.com/antonio2368)). +* Improve logging macros [#52519](https://github.com/ClickHouse/ClickHouse/pull/52519) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `toDecimalString` function [#52520](https://github.com/ClickHouse/ClickHouse/pull/52520) ([Andrey Zvonov](https://github.com/zvonand)). +* Remove unused code [#52527](https://github.com/ClickHouse/ClickHouse/pull/52527) ([Raúl Marín](https://github.com/Algunenano)). +* Cancel execution in PipelineExecutor in case of exception in graph->updateNode [#52533](https://github.com/ClickHouse/ClickHouse/pull/52533) ([Kruglov Pavel](https://github.com/Avogar)). +* Make 01951_distributed_push_down_limit analyzer agnostic [#52534](https://github.com/ClickHouse/ClickHouse/pull/52534) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix disallow_concurrency test for backup and restore [#52536](https://github.com/ClickHouse/ClickHouse/pull/52536) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update 02136_scalar_subquery_metrics.sql [#52537](https://github.com/ClickHouse/ClickHouse/pull/52537) ([Alexander Tokmakov](https://github.com/tavplubix)). +* tests: fix 01035_avg_weighted_long flakiness [#52556](https://github.com/ClickHouse/ClickHouse/pull/52556) ([Azat Khuzhin](https://github.com/azat)). +* tests: increase throttling for 01923_network_receive_time_metric_insert [#52557](https://github.com/ClickHouse/ClickHouse/pull/52557) ([Azat Khuzhin](https://github.com/azat)). +* tests: fix 00719_parallel_ddl_table flakiness in debug builds [#52558](https://github.com/ClickHouse/ClickHouse/pull/52558) ([Azat Khuzhin](https://github.com/azat)). +* tests: fix 01821_join_table_race_long flakiness [#52559](https://github.com/ClickHouse/ClickHouse/pull/52559) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky `00995_exception_while_insert` [#52568](https://github.com/ClickHouse/ClickHouse/pull/52568) ([Antonio Andelic](https://github.com/antonio2368)). +* MaterializedMySQL: Fix typos in tests [#52575](https://github.com/ClickHouse/ClickHouse/pull/52575) ([Val Doroshchuk](https://github.com/valbok)). +* Fix `02497_trace_events_stress_long` again [#52587](https://github.com/ClickHouse/ClickHouse/pull/52587) ([Antonio Andelic](https://github.com/antonio2368)). +* Revert "Remove `mmap/mremap/munmap` from Allocator.h" [#52589](https://github.com/ClickHouse/ClickHouse/pull/52589) ([Nikita Taranov](https://github.com/nickitat)). +* Remove peak memory usage from the final message in the client [#52598](https://github.com/ClickHouse/ClickHouse/pull/52598) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* GinIndexStore: fix a bug when files are finalizated after first write, [#52602](https://github.com/ClickHouse/ClickHouse/pull/52602) ([Sema Checherinda](https://github.com/CheSema)). +* Fix deadlocks in StorageTableFunctionProxy [#52626](https://github.com/ClickHouse/ClickHouse/pull/52626) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix build with clang-15 [#52627](https://github.com/ClickHouse/ClickHouse/pull/52627) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix style [#52647](https://github.com/ClickHouse/ClickHouse/pull/52647) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix logging level of a noisy message [#52648](https://github.com/ClickHouse/ClickHouse/pull/52648) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Revert "Added field `refcount` to `system.remote_data_paths` table" [#52657](https://github.com/ClickHouse/ClickHouse/pull/52657) ([Alexander Tokmakov](https://github.com/tavplubix)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8b535e3d897..1eabc65a10f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.7.1.2470-stable 2023-07-27 v23.6.2.18-stable 2023-07-09 v23.6.1.1524-stable 2023-06-30 v23.5.4.25-stable 2023-06-29 From ce38d3c5ea45507696430e9c7f39f9ab7b9de394 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:11:08 +0200 Subject: [PATCH 47/72] address comment --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 53481ab06a0..dbb4f7f0d8e 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,11 +10,17 @@ #include #include +#include "Common/Exception.h" #include namespace DB { +namespace ErrorCodes +{ + extern const int ABORTED; +} + namespace { @@ -271,8 +277,7 @@ void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_inf if (stopped) { - LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we drop this part."); - return; + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); } chassert(current_writer != nullptr); From f5dfb70f5c2f4b94a54e9fdb97737a70b28362ad Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:12:20 +0200 Subject: [PATCH 48/72] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index dbb4f7f0d8e..80e94b2fd39 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,7 +10,7 @@ #include #include -#include "Common/Exception.h" +#include #include namespace DB From 0d44d527ef590a5471ea577c132edb42f0c99c70 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:27:04 +0200 Subject: [PATCH 49/72] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 80e94b2fd39..25b93160d27 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -241,8 +241,7 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: if (stopped) { - LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we add this part."); - return {}; + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); } chassert(current_writer != nullptr); From f6ca013c536d76ca6c1403db5f84d792f6bd8864 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:28:00 +0200 Subject: [PATCH 50/72] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 25b93160d27..22dabc43a8c 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -11,7 +11,6 @@ #include #include -#include namespace DB { From 9488567bf6be7e2b751917a179222478fcb46f5e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:28:08 +0200 Subject: [PATCH 51/72] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 22dabc43a8c..548b61ce422 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -240,7 +240,7 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: if (stopped) { - throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we add this part."); } chassert(current_writer != nullptr); From 5611b2fff484d74c70c8ad6b62ba8d66c0b63589 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 27 Jul 2023 15:45:55 +0200 Subject: [PATCH 52/72] Add a note about not working _table filter for Merge with analyzer Signed-off-by: Azat Khuzhin --- src/Storages/StorageMerge.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index b0ed242d14d..272f35303bd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -795,6 +795,10 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( bool filter_by_database_virtual_column /* = false */, bool filter_by_table_virtual_column /* = false */) const { + /// FIXME: filtering does not work with allow_experimental_analyzer due to + /// different column names there (it has "table_name._table" not just + /// "_table") + assert(!filter_by_database_virtual_column || !filter_by_table_virtual_column || query); const Settings & settings = query_context->getSettingsRef(); From 68aed0d16e331a6ba6b592243f10ce2a816152db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 26 Jul 2023 20:25:48 +0200 Subject: [PATCH 53/72] RFC: Fix filtering by virtual columns with OR expression Virtual columns did not supports queries with OR, for example query like this (here `m` is the `Merge` table, see the test): select key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1'); Will always leads to: Cannot find column `value` in source stream, there are only columns ... The reason for this is that it actually executes the following queries: SELECT key, value FROM default.d1 WHERE ((value = 10) AND ('v1' = 'v1')) OR ((value = 20) AND ('v1' = 'v1')); SELECT key FROM default.d2 WHERE 0; And this kind of filtering is used not only for `Merge` table but also: - `_table` for `Merge` (already mentioned) - `_file` for `File` - `_idx` for `S3` - and as well as filtering `system.*` tables by `database`/`table`/... Signed-off-by: Azat Khuzhin --- src/Storages/VirtualColumnUtils.cpp | 36 +++++++++++++----- .../02840_merge__table_or_filter.reference | 38 +++++++++++++++++++ .../02840_merge__table_or_filter.sql.j2 | 34 +++++++++++++++++ 3 files changed, 99 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02840_merge__table_or_filter.reference create mode 100644 tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 907fc0cd22c..79be1f98a0f 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -63,14 +64,31 @@ bool isValidFunction(const ASTPtr & expression, const std::function & is_constant, ASTs & result) { const auto * function = expression->as(); - if (function && (function->name == "and" || function->name == "indexHint")) + + if (function) { - bool ret = true; - for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, is_constant, result); - return ret; + if (function->name == "and" || function->name == "indexHint") + { + bool ret = true; + for (const auto & child : function->arguments->children) + ret &= extractFunctions(child, is_constant, result); + return ret; + } + else if (function->name == "or") + { + bool ret = true; + ASTs or_args; + for (const auto & child : function->arguments->children) + ret &= extractFunctions(child, is_constant, or_args); + /// We can keep condition only if it still OR condition (i.e. we + /// have dependent conditions for columns at both sides) + if (or_args.size() == 2) + result.push_back(makeASTForLogicalOr(std::move(or_args))); + return ret; + } } - else if (isValidFunction(expression, is_constant)) + + if (isValidFunction(expression, is_constant)) { result.push_back(expression->clone()); return true; @@ -80,13 +98,13 @@ bool extractFunctions(const ASTPtr & expression, const std::function Date: Thu, 27 Jul 2023 09:49:34 +0000 Subject: [PATCH 54/72] Add query cache metrics to system.asynchronous_metrics Cf. https://github.com/ClickHouse/ClickHouse/pull/52384#issuecomment-1653241216 --- docs/en/operations/query-cache.md | 11 ++++++----- .../operations/system-tables/asynchronous_metrics.md | 12 ++++++++++++ docs/en/operations/system-tables/events.md | 2 ++ docs/en/operations/system-tables/metrics.md | 2 +- src/Interpreters/Cache/QueryCache.cpp | 10 ++++++++++ src/Interpreters/Cache/QueryCache.h | 5 ++++- src/Interpreters/ServerAsynchronousMetrics.cpp | 6 ++++++ 7 files changed, 41 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 547105c65cc..d0b785d8fda 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -61,11 +61,12 @@ use_query_cache = true`) but one should keep in mind that all `SELECT` queries i may return cached results then. The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table -`system.query_cache`. The number of query cache hits and misses are shown as events "QueryCacheHits" and "QueryCacheMisses" in system table -[system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = -true". Other queries do not affect the cache miss counter. Field `query_log_usage` in system table -[system.query_log](system-tables/query_log.md) shows for each ran query whether the query result was written into or read from the query -cache. +`system.query_cache`. The number of query cache hits and misses since database start are shown as events "QueryCacheHits" and +"QueryCacheMisses" in system table [system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run +with setting `use_query_cache = true`, other queries do not affect "QueryCacheMisses". Field `query_log_usage` in system table +[system.query_log](system-tables/query_log.md) shows for each executed query whether the query result was written into or read from the +query cache. Asynchronous metrics "QueryCacheEntries" and "QueryCacheBytes" in system table +[system.asynchronous_metrics](system-tables/asynchronous_metrics.md) show how many entries / bytes the query cache currently contains. The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be changed (see below) but doing so is not recommended for security reasons. diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index f357341da67..e46b495239c 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -32,6 +32,10 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 └─────────────────────────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` + + ## Metric descriptions @@ -483,6 +487,14 @@ The value is similar to `OSUserTime` but divided to the number of CPU cores to b Number of threads in the server of the PostgreSQL compatibility protocol. +### QueryCacheBytes + +Total size of the query cache cache in bytes. + +### QueryCacheEntries + +Total number of entries in the query cache. + ### ReplicasMaxAbsoluteDelay Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data. diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index ba5602ee292..7846fe4be5d 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -11,6 +11,8 @@ Columns: - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. - `description` ([String](../../sql-reference/data-types/string.md)) — Event description. +You can find all supported events in source file [src/Common/ProfileEvents.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ProfileEvents.cpp). + **Example** ``` sql diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 5a7dfd03eb4..b1dcea5500f 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -11,7 +11,7 @@ Columns: - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. - `description` ([String](../../sql-reference/data-types/string.md)) — Metric description. -The list of supported metrics you can find in the [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) source file of ClickHouse. +You can find all supported metrics in source file [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp). **Example** diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index a6c509e8bb1..5982a5ade50 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -496,6 +496,16 @@ void QueryCache::reset() cache_size_in_bytes = 0; } +size_t QueryCache::weight() const +{ + return cache.weight(); +} + +size_t QueryCache::count() const +{ + return cache.count(); +} + size_t QueryCache::recordQueryRun(const Key & key) { std::lock_guard lock(mutex); diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index c2de8ca22dd..eaa54c503fa 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -186,6 +186,9 @@ public: void reset(); + size_t weight() const; + size_t count() const; + /// Record new execution of query represented by key. Returns number of executions so far. size_t recordQueryRun(const Key & key); @@ -193,7 +196,7 @@ public: std::vector dump() const; private: - Cache cache; + Cache cache; /// has its own locking --> not protected by mutex mutable std::mutex mutex; TimesExecuted times_executed TSA_GUARDED_BY(mutex); diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 0fbcfc9e6a1..68411e80755 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -92,6 +92,12 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values " The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."}; } + if (auto query_cache = getContext()->getQueryCache()) + { + new_values["QueryCacheBytes"] = { query_cache->weight(), "Total size of the query cache in bytes." }; + new_values["QueryCacheEntries"] = { query_cache->count(), "Total number of entries in the query cache." }; + } + { auto caches = FileCacheFactory::instance().getAll(); size_t total_bytes = 0; From 380da315121078fc3e88a1e038e5aacd296853c2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 27 Jul 2023 19:03:44 +0200 Subject: [PATCH 55/72] Improvements to backup restore disallow_concurrency test --- .../test_disallow_concurrency.py | 102 +++++++++++++----- 1 file changed, 73 insertions(+), 29 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index a863a6e2047..af1b2656227 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -133,21 +133,31 @@ def test_concurrent_backups_on_same_node(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - try: - error = nodes[0].query_and_get_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) - except Exception as e: + result, error = nodes[0].query_and_get_answer_with_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ) + + if not error: status = ( nodes[0] .query(f"SELECT status FROM system.backups WHERE id == '{id}'") .rstrip("\n") ) # It is possible that the second backup was picked up first, and then the async backup - if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + if status == "BACKUP_FAILED": + return + elif status == "CREATING_BACKUP": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id = '{id}'", + "BACKUP_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent backups both passed, when one is expected to fail") + expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -191,20 +201,31 @@ def test_concurrent_backups_on_different_nodes(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - try: - error = nodes[0].query_and_get_error( + result, error = nodes[0].query_and_get_answer_with_error( f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) - except Exception as e: + ) + + if not error: status = ( nodes[1] .query(f"SELECT status FROM system.backups WHERE id == '{id}'") .rstrip("\n") ) - if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "BACKUP_FAILED": + return + elif status == "CREATING_BACKUP": + assert_eq_with_retry( + nodes[1], + f"SELECT status FROM system.backups WHERE id = '{id}'", + "BACKUP_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent backups both passed, when one is expected to fail") + expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -247,20 +268,32 @@ def test_concurrent_restores_on_same_node(): ) assert status in ["RESTORING", "RESTORED"] - try: - error = nodes[0].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) - except Exception as e: + result, error = nodes[0].query_and_get_answer_with_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + + if not error: status = ( nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") .rstrip("\n") ) - if status == "RESTORING" or status == "RESTORE_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "RESTORE_FAILED": + return + elif status == "RESTORING": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id == '{restore_id}'", + "RESTORE_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent restores both passed, when one is expected to fail") + + expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", @@ -303,20 +336,31 @@ def test_concurrent_restores_on_different_node(): ) assert status in ["RESTORING", "RESTORED"] - try: - error = nodes[1].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) - except Exception as e: + result, error = nodes[1].query_and_get_answer_with_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + + if not error: status = ( nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") .rstrip("\n") ) - if status == "RESTORING" or status == "RESTORE_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "RESTORE_FAILED": + return + elif status == "RESTORING": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id == '{restore_id}'", + "RESTORE_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent restores both passed, when one is expected to fail") + expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", From b95745d916330abb3306016d512b40d3d24616dd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 17:52:44 +0000 Subject: [PATCH 56/72] fix: check positional options --- src/Client/ClientBase.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 496fc8fce0a..06dabf96c28 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2624,6 +2624,10 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); } + /// Check positional options. + if (std::ranges::count_if(parsed.options, [](const auto & op){ return !op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--"); }) > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); + po::store(parsed, options); } From 9340f02d26ae7f170611ea9b19a11e720b41b765 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 18:33:07 +0000 Subject: [PATCH 57/72] Silence spell check --- .../aspell-ignore/en/aspell-dict.txt | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..80aeadd8738 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -211,7 +211,6 @@ Decrypted Deduplicate Deduplication DelayedInserts -delim DeliveryTag DeltaLake Denormalize @@ -699,6 +698,8 @@ PyCharm QEMU QTCreator Quantile +QueryCacheBytes +QueryCacheEntries QueryCacheHits QueryCacheMisses QueryPreempted @@ -761,9 +762,9 @@ RoaringBitmap RocksDB Rollup RowBinary +RowBinaryWithDefaults RowBinaryWithNames RowBinaryWithNamesAndTypes -RowBinaryWithDefaults Runtime SATA SELECTs @@ -776,7 +777,6 @@ SMALLINT SPNEGO SQEs SQLAlchemy -SquaredDistance SQLConsoleDetail SQLInsert SQLSTATE @@ -811,6 +811,7 @@ Smirnov'test Soundex SpanKind Spearman's +SquaredDistance StartTLS StartTime StartupSystemTables @@ -838,8 +839,6 @@ Subexpression Submodules Subqueries Substrings -substringIndex -substringIndexUTF SummingMergeTree SuperSet Superset @@ -1272,6 +1271,7 @@ cryptographic csv csvwithnames csvwithnamesandtypes +curdate currentDatabase currentProfiles currentRoles @@ -1331,6 +1331,7 @@ defaultProfiles defaultRoles defaultValueOfArgumentType defaultValueOfTypeName +delim deltaLake deltaSum deltaSumTimestamp @@ -1542,13 +1543,13 @@ hadoop halfMD halfday hardlinks +hasAll +hasAny +hasColumnInTable hasSubsequence hasSubsequenceCaseInsensitive hasSubsequenceCaseInsensitiveUTF hasSubsequenceUTF -hasAll -hasAny -hasColumnInTable hasSubstr hasToken hasTokenCaseInsensitive @@ -1590,10 +1591,10 @@ incrementing indexHint indexOf infi -initialQueryID -initializeAggregation initcap initcapUTF +initialQueryID +initializeAggregation injective innogames inodes @@ -2131,9 +2132,9 @@ routineley rowNumberInAllBlocks rowNumberInBlock rowbinary +rowbinarywithdefaults rowbinarywithnames rowbinarywithnamesandtypes -rowbinarywithdefaults rsync rsyslog runnable @@ -2185,8 +2186,8 @@ sleepEachRow snowflakeToDateTime socketcache soundex -sparkbar sparkBar +sparkbar sparsehash speedscope splitByChar @@ -2256,6 +2257,8 @@ subreddits subseconds subsequence substring +substringIndex +substringIndexUTF substringUTF substrings subtitiles @@ -2556,4 +2559,3 @@ znode znodes zookeeperSessionUptime zstd -curdate From 5942c80faed38febea5394526b5e5c670b03bd4d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 19:11:44 +0000 Subject: [PATCH 58/72] fix test: '--option' now is allowed after terminating '--' --- .../02096_bad_options_in_client_and_local.reference | 2 -- .../0_stateless/02096_bad_options_in_client_and_local.sh | 4 ---- 2 files changed, 6 deletions(-) diff --git a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference index c4c0901b9df..432299e9556 100644 --- a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference +++ b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference @@ -8,5 +8,3 @@ OK OK OK OK -OK -OK diff --git a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh index d37155e8506..753d56fb424 100755 --- a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh +++ b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh @@ -9,8 +9,6 @@ ${CLICKHOUSE_LOCAL} --unknown-option 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" ${CLICKHOUSE_LOCAL} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" -${CLICKHOUSE_LOCAL} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" - ${CLICKHOUSE_LOCAL} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" ${CLICKHOUSE_LOCAL} -f 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" @@ -22,8 +20,6 @@ ${CLICKHOUSE_CLIENT} --unknown-option 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" ${CLICKHOUSE_CLIENT} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" -${CLICKHOUSE_CLIENT} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" - ${CLICKHOUSE_CLIENT} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" ${CLICKHOUSE_CLIENT} --j 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" From dba8b445bd37b2fb9fb4983e0a3f740649dcbb5b Mon Sep 17 00:00:00 2001 From: Jai Jhala Date: Thu, 27 Jul 2023 12:32:53 -0700 Subject: [PATCH 59/72] Update default output_format_arrow_compression.md Updates the default parameter of output_format_arrow_compression_method from "none" to "lz4_frame". --- docs/en/operations/settings/settings-formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index ee8e0d547b8..fb10ff7f61b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1164,7 +1164,7 @@ Enabled by default. Compression method used in output Arrow format. Supported codecs: `lz4_frame`, `zstd`, `none` (uncompressed) -Default value: `none`. +Default value: `lz4_frame`. ## ORC format settings {#orc-format-settings} From 7d8dc92ed0522e309760037720f6fd8fb3f2542d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Jul 2023 21:07:54 +0000 Subject: [PATCH 60/72] Automatic style fix --- .../test_disallow_concurrency.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index af1b2656227..5c3f06a9d9d 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -156,7 +156,9 @@ def test_concurrent_backups_on_same_node(): ) return else: - raise Exception("Concurrent backups both passed, when one is expected to fail") + raise Exception( + "Concurrent backups both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent backups not supported", @@ -202,7 +204,7 @@ def test_concurrent_backups_on_different_nodes(): assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] result, error = nodes[0].query_and_get_answer_with_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" ) if not error: @@ -224,7 +226,9 @@ def test_concurrent_backups_on_different_nodes(): ) return else: - raise Exception("Concurrent backups both passed, when one is expected to fail") + raise Exception( + "Concurrent backups both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent backups not supported", @@ -291,8 +295,9 @@ def test_concurrent_restores_on_same_node(): ) return else: - raise Exception("Concurrent restores both passed, when one is expected to fail") - + raise Exception( + "Concurrent restores both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent restores not supported", @@ -359,7 +364,9 @@ def test_concurrent_restores_on_different_node(): ) return else: - raise Exception("Concurrent restores both passed, when one is expected to fail") + raise Exception( + "Concurrent restores both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent restores not supported", From 18c1fd6f08cc2be964ed15604c26a70d7d168561 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 27 Jul 2023 21:24:39 +0000 Subject: [PATCH 61/72] Refactor InDepthQueryTreeVisitorWithContext --- src/Analyzer/InDepthQueryTreeVisitor.h | 158 +++--------------- ...egateFunctionsArithmericOperationsPass.cpp | 8 +- src/Analyzer/Passes/ArrayExistsToHasPass.cpp | 2 +- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 2 +- .../Passes/ConvertOrLikeChainPass.cpp | 2 +- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 2 +- src/Analyzer/Passes/CountDistinctPass.cpp | 34 ++-- src/Analyzer/Passes/CrossToInnerJoinPass.cpp | 2 +- .../Passes/FunctionToSubcolumnsPass.cpp | 2 +- src/Analyzer/Passes/FuseFunctionsPass.cpp | 2 +- .../Passes/GroupingFunctionsResolvePass.cpp | 2 +- src/Analyzer/Passes/IfChainToMultiIfPass.cpp | 2 +- .../Passes/IfTransformStringsToEnumPass.cpp | 2 +- .../Passes/LogicalExpressionOptimizerPass.cpp | 2 +- src/Analyzer/Passes/MultiIfToIfPass.cpp | 2 +- .../Passes/NormalizeCountVariantsPass.cpp | 2 +- .../OptimizeGroupByFunctionKeysPass.cpp | 2 +- ...ptimizeRedundantFunctionsInOrderByPass.cpp | 2 +- .../RewriteAggregateFunctionWithIfPass.cpp | 2 +- .../Passes/ShardNumColumnToFunctionPass.cpp | 2 +- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 2 +- .../UniqInjectiveFunctionsEliminationPass.cpp | 2 +- src/Storages/buildQueryTreeForShard.cpp | 2 +- 23 files changed, 55 insertions(+), 185 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index be3a760d4e6..59ee57996c4 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -91,26 +91,25 @@ private: template using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor; -/** Same as InDepthQueryTreeVisitor and additionally keeps track of current scope context. +/** Same as InDepthQueryTreeVisitor (but has a different interface) and additionally keeps track of current scope context. * This can be useful if your visitor has special logic that depends on current scope context. + * + * To specify behavior of the visitor you can implement following methods in derived class: + * 1. needChildVisit – This methods allows to skip subtree. + * 2. enterImpl – This method is called before children are processed. + * 3. leaveImpl – This method is called after children are processed. */ template class InDepthQueryTreeVisitorWithContext { public: - using VisitQueryTreeNodeType = std::conditional_t; + using VisitQueryTreeNodeType = QueryTreeNodePtr; explicit InDepthQueryTreeVisitorWithContext(ContextPtr context, size_t initial_subquery_depth = 0) : current_context(std::move(context)) , subquery_depth(initial_subquery_depth) {} - /// Return true if visitor should traverse tree top to bottom, false otherwise - bool shouldTraverseTopToBottom() const - { - return true; - } - /// Return true if visitor should visit child, false otherwise bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]]) { @@ -147,18 +146,16 @@ public: ++subquery_depth; - bool traverse_top_to_bottom = getDerived().shouldTraverseTopToBottom(); - if (!traverse_top_to_bottom) - visitChildren(query_tree_node); + getDerived().enterImpl(query_tree_node); - getDerived().visitImpl(query_tree_node); - - if (traverse_top_to_bottom) - visitChildren(query_tree_node); + visitChildren(query_tree_node); getDerived().leaveImpl(query_tree_node); } + void enterImpl(VisitQueryTreeNodeType & node [[maybe_unused]]) + {} + void leaveImpl(VisitQueryTreeNodeType & node [[maybe_unused]]) {} private: @@ -172,85 +169,15 @@ private: return *static_cast(this); } - void visitChildren(VisitQueryTreeNodeType & expression) + bool shouldSkipSubtree( + VisitQueryTreeNodeType & parent, + VisitQueryTreeNodeType & child, + size_t subtree_index) { - for (auto & child : expression->getChildren()) - { - if (!child) - continue; + bool need_visit_child = getDerived().needChildVisit(parent, child); + if (!need_visit_child) + return true; - bool need_visit_child = getDerived().needChildVisit(expression, child); - - if (need_visit_child) - visit(child); - } - } - - ContextPtr current_context; - size_t subquery_depth = 0; -}; - -template -using ConstInDepthQueryTreeVisitorWithContext = InDepthQueryTreeVisitorWithContext; - -/** Visitor that use another visitor to visit node only if condition for visiting node is true. - * For example, your visitor need to visit only query tree nodes or union nodes. - * - * Condition interface: - * struct Condition - * { - * bool operator()(VisitQueryTreeNodeType & node) - * { - * return shouldNestedVisitorVisitNode(node); - * } - * } - */ -template -class InDepthQueryTreeConditionalVisitor : public InDepthQueryTreeVisitor, const_visitor> -{ -public: - using Base = InDepthQueryTreeVisitor, const_visitor>; - using VisitQueryTreeNodeType = typename Base::VisitQueryTreeNodeType; - - explicit InDepthQueryTreeConditionalVisitor(Visitor & visitor_, Condition & condition_) - : visitor(visitor_) - , condition(condition_) - { - } - - bool shouldTraverseTopToBottom() const - { - return visitor.shouldTraverseTopToBottom(); - } - - void visitImpl(VisitQueryTreeNodeType & query_tree_node) - { - if (condition(query_tree_node)) - visitor.visit(query_tree_node); - } - - Visitor & visitor; - Condition & condition; -}; - -template -using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor; - -template -class QueryTreeVisitor -{ -public: - explicit QueryTreeVisitor(ContextPtr context_) - : current_context(std::move(context_)) - {} - - bool needApply(QueryTreeNodePtr & node) - { - return getImpl().needApply(node); - } - - bool shouldSkipSubtree(QueryTreeNodePtr & parent, size_t subtree_index) - { if (auto * table_function_node = parent->as()) { const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes(); @@ -259,58 +186,19 @@ public: return false; } - void visit(QueryTreeNodePtr & node) - { - auto current_scope_context_ptr = current_context; - SCOPE_EXIT( - current_context = std::move(current_scope_context_ptr); - ); - - if (auto * query_node = node->template as()) - current_context = query_node->getContext(); - else if (auto * union_node = node->template as()) - current_context = union_node->getContext(); - - if (!TOP_TO_BOTTOM) - visitChildren(node); - - if (needApply(node)) - getImpl().apply(node); - - if (TOP_TO_BOTTOM) - visitChildren(node); - } - - const ContextPtr & getContext() const - { - return current_context; - } - - const Settings & getSettings() const - { - return current_context->getSettingsRef(); - } -private: - - Impl & getImpl() - { - return *static_cast(this); - } - - void visitChildren(QueryTreeNodePtr & node) + void visitChildren(VisitQueryTreeNodeType & expression) { size_t index = 0; - for (auto & child : node->getChildren()) + for (auto & child : expression->getChildren()) { - if (child && !shouldSkipSubtree(node, index)) + if (child && !shouldSkipSubtree(expression, child, index)) visit(child); ++index; } } - static constexpr bool TOP_TO_BOTTOM = Impl::TOP_TO_BOTTOM; - ContextPtr current_context; + size_t subquery_depth = 0; }; } diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index 1476a66c892..3615a632374 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -51,13 +51,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - /// Traverse tree bottom to top - static bool shouldTraverseTopToBottom() - { - return false; - } - - void visitImpl(QueryTreeNodePtr & node) + void leaveImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions) return; diff --git a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp index c0f958588f1..a95bcea4fac 100644 --- a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp +++ b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp @@ -22,7 +22,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_array_exists_to_has) return; diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index 15326ca1dc8..2c89ec9dc20 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -20,7 +20,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().final) return; diff --git a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp index 7d7362fb742..1fada88a21c 100644 --- a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp +++ b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp @@ -50,7 +50,7 @@ public: && settings.max_hyperscan_regexp_total_length == 0; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "or") diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 4d32c96b845..724448ad742 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -688,7 +688,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * query_node = node->as(); if (!query_node) diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 38f7d07d052..dc58747221e 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -16,17 +16,16 @@ namespace DB namespace { -class CountDistinctVisitor : public QueryTreeVisitor +class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContext { public: - using QueryTreeVisitor::QueryTreeVisitor; + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; - static constexpr bool TOP_TO_BOTTOM = true; - - bool needApply(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().count_distinct_optimization) - return false; + return; auto * query_node = node->as(); @@ -34,43 +33,32 @@ public: if (!query_node || (query_node->hasWith() || query_node->hasPrewhere() || query_node->hasWhere() || query_node->hasGroupBy() || query_node->hasHaving() || query_node->hasWindow() || query_node->hasOrderBy() || query_node->hasLimitByLimit() || query_node->hasLimitByOffset() || query_node->hasLimitBy() || query_node->hasLimit() || query_node->hasOffset())) - return false; + return; /// Check that query has only single table expression auto join_tree_node_type = query_node->getJoinTree()->getNodeType(); if (join_tree_node_type == QueryTreeNodeType::JOIN || join_tree_node_type == QueryTreeNodeType::ARRAY_JOIN) - return false; + return; /// Check that query has only single node in projection auto & projection_nodes = query_node->getProjection().getNodes(); if (projection_nodes.size() != 1) - return false; + return; /// Check that query single projection node is `countDistinct` function auto & projection_node = projection_nodes[0]; auto * function_node = projection_node->as(); if (!function_node) - return false; + return; auto lower_function_name = Poco::toLower(function_node->getFunctionName()); if (lower_function_name != "countdistinct" && lower_function_name != "uniqexact") - return false; + return; /// Check that `countDistinct` function has single COLUMN argument auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) - return false; - - return true; - } - - void apply(QueryTreeNodePtr & node) - { - auto * query_node = node->as(); - auto & projection_nodes = query_node->getProjection().getNodes(); - auto * function_node = projection_nodes[0]->as(); - - auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); + return; auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp index d4877d23f28..b5ece1a4c49 100644 --- a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp +++ b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp @@ -193,7 +193,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!isEnabled()) return; diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 696483862e0..cd635f87e0e 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -29,7 +29,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) const + void enterImpl(QueryTreeNodePtr & node) const { if (!getSettings().optimize_functions_to_subcolumns) return; diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index 14082697955..2cb7afa4ad6 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -37,7 +37,7 @@ public: , names_to_collect(names_to_collect_) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_syntax_fuse_functions) return; diff --git a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp index 0cf5310a3ad..577bca8d1ae 100644 --- a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp +++ b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp @@ -46,7 +46,7 @@ public: { } - void visitImpl(const QueryTreeNodePtr & node) + void enterImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "grouping") diff --git a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp index 1f97e012331..b0018d474d5 100644 --- a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp +++ b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp @@ -23,7 +23,7 @@ public: , multi_if_function_ptr(std::move(multi_if_function_ptr_)) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_if_chain_to_multiif) return; diff --git a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp index 562aff4cf05..901867b8889 100644 --- a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp +++ b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp @@ -113,7 +113,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_if_transform_strings_to_enum) return; diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 13f8025f5ea..46056aeaf6f 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -19,7 +19,7 @@ public: : Base(std::move(context)) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); diff --git a/src/Analyzer/Passes/MultiIfToIfPass.cpp b/src/Analyzer/Passes/MultiIfToIfPass.cpp index 4672351bcfb..85dd33af8bb 100644 --- a/src/Analyzer/Passes/MultiIfToIfPass.cpp +++ b/src/Analyzer/Passes/MultiIfToIfPass.cpp @@ -21,7 +21,7 @@ public: , if_function_ptr(std::move(if_function_ptr_)) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_multiif_to_if) return; diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index d36be98751c..c85b863a203 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -20,7 +20,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_normalize_count_variants) return; diff --git a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp index 5ed52f1210b..2e3f207fdeb 100644 --- a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp @@ -26,7 +26,7 @@ public: return !child->as(); } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_group_by_function_keys) return; diff --git a/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp b/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp index c6d312d0ecf..875d0c8b5fb 100644 --- a/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp +++ b/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp @@ -28,7 +28,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_redundant_functions_in_order_by) return; diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index de264948d4c..38f2fbfa274 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -26,7 +26,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_aggregate_function_with_if) return; diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp index b28816e8ff3..52c30b7b35d 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -24,7 +24,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) const + void enterImpl(QueryTreeNodePtr & node) const { auto * column_node = node->as(); if (!column_node) diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index d55af278152..cff9ba1111c 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -26,7 +26,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_sum_if_to_count_if) return; diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp index 5c4484457e8..179bd1c38e4 100644 --- a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp @@ -31,7 +31,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_injective_functions_inside_uniq) return; diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 1ee7d747fcc..9929b5bb39b 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -130,7 +130,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); auto * join_node = node->as(); From 6573ba537819ce03dd644ff02bdf7341bcc26d58 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 19:37:28 -0400 Subject: [PATCH 62/72] Temporary returning metadata_cache.xml into tests config --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 9aaadbc74a5..50f2627d37c 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -34,6 +34,7 @@ ln -sf $SRC_PATH/config.d/keeper_port.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/merge_tree.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/lost_forever_check.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/metadata_cache.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/prometheus.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ From b3351bb547b8753b405d820925f8f4270be6132d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 28 Jul 2023 03:36:23 +0000 Subject: [PATCH 63/72] partially fixed 01747_system_session_log_long test --- src/Core/PostgreSQLProtocol.h | 49 +++-- tests/config/users.d/session_log_test.xml | 2 +- .../01747_system_session_log_long.reference | 198 ++++++++++++------ .../01747_system_session_log_long.sh | 119 ++++++----- 4 files changed, 224 insertions(+), 144 deletions(-) rename tests/queries/{bugs => 0_stateless}/01747_system_session_log_long.reference (73%) rename tests/queries/{bugs => 0_stateless}/01747_system_session_log_long.sh (78%) diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 8c0654b559f..b0d7646a5f7 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -805,20 +805,9 @@ protected: const String & user_name, const String & password, Session & session, - Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - try - { - session.authenticate(user_name, password, address); - } - catch (const Exception &) - { - mt.send( - Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "28P01", "Invalid user or password"), - true); - throw; - } + session.authenticate(user_name, password, address); } public: @@ -839,10 +828,10 @@ public: void authenticate( const String & user_name, Session & session, - Messaging::MessageTransport & mt, + [[maybe_unused]] Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) override { - return setPassword(user_name, "", session, mt, address); + return setPassword(user_name, "", session, address); } AuthenticationType getType() const override @@ -866,7 +855,7 @@ public: if (type == Messaging::FrontMessageType::PASSWORD_MESSAGE) { std::unique_ptr password = mt.receive(); - return setPassword(user_name, password->password, session, mt, address); + return setPassword(user_name, password->password, session, address); } else throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, @@ -901,20 +890,30 @@ public: Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - const AuthenticationType user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); - if (type_to_method.find(user_auth_type) != type_to_method.end()) + AuthenticationType user_auth_type; + try { - type_to_method[user_auth_type]->authenticate(user_name, session, mt, address); - mt.send(Messaging::AuthenticationOk(), true); - LOG_DEBUG(log, "Authentication for user {} was successful.", user_name); - return; + user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); + if (type_to_method.find(user_auth_type) != type_to_method.end()) + { + type_to_method[user_auth_type]->authenticate(user_name, session, mt, address); + mt.send(Messaging::AuthenticationOk(), true); + LOG_DEBUG(log, "Authentication for user {} was successful.", user_name); + return; + } + } + catch (const Exception&) + { + mt.send(Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "28P01", "Invalid user or password"), + true); + + throw; } - mt.send( - Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "0A000", "Authentication method is not supported"), - true); + mt.send(Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "0A000", "Authentication method is not supported"), + true); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Authentication type {} is not supported.", user_auth_type); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Authentication method is not supported: {}", user_auth_type); } }; } diff --git a/tests/config/users.d/session_log_test.xml b/tests/config/users.d/session_log_test.xml index daddaa6e4b9..cc2c2c5fcde 100644 --- a/tests/config/users.d/session_log_test.xml +++ b/tests/config/users.d/session_log_test.xml @@ -17,7 +17,7 @@ - + ::1 127.0.0.1 diff --git a/tests/queries/bugs/01747_system_session_log_long.reference b/tests/queries/0_stateless/01747_system_session_log_long.reference similarity index 73% rename from tests/queries/bugs/01747_system_session_log_long.reference rename to tests/queries/0_stateless/01747_system_session_log_long.reference index 9ecf7e05421..e4f0b6f6076 100644 --- a/tests/queries/bugs/01747_system_session_log_long.reference +++ b/tests/queries/0_stateless/01747_system_session_log_long.reference @@ -4,215 +4,291 @@ TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - No profiles no roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - Two profiles, no roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - Two profiles and two simple roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # plaintext_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # plaintext_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # plaintext_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # sha256_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # sha256_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # sha256_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # double_sha1_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password # double_sha1_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password # double_sha1_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL Logout 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_two_profiles_two_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles MySQL LoginFailure many invalid_session_log_test_xml_user TCP LoginFailure 1 -invalid_session_log_test_xml_user HTTP LoginFailure 1 +invalid_session_log_test_xml_user HTTP LoginFailure many invalid_session_log_test_xml_user MySQL LoginFailure many +invalid_session_log_test_xml_user PostgreSQL LoginFailure many session_log_test_xml_user TCP LoginSuccess 1 session_log_test_xml_user TCP Logout 1 -session_log_test_xml_user HTTP LoginSuccess 1 -session_log_test_xml_user HTTP Logout 1 +session_log_test_xml_user HTTP LoginSuccess many +session_log_test_xml_user HTTP Logout many session_log_test_xml_user MySQL LoginSuccess 1 session_log_test_xml_user MySQL Logout 1 diff --git a/tests/queries/bugs/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh similarity index 78% rename from tests/queries/bugs/01747_system_session_log_long.sh rename to tests/queries/0_stateless/01747_system_session_log_long.sh index 9b127e0b48d..c6e93f4abd7 100755 --- a/tests/queries/bugs/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash # Tags: long, no-parallel, no-fasttest -# Tag no-fasttest: Accesses CH via mysql table function (which is unavailable) ################################################################################################## # Verify that login, logout, and login failure events are properly stored in system.session_log @@ -11,9 +10,8 @@ # Using multiple protocols # * native TCP protocol with CH client # * HTTP with CURL -# * MySQL - CH server accesses itself via mysql table function, query typically fails (unrelated) -# but auth should be performed properly. -# * PostgreSQL - CH server accesses itself via postgresql table function (currently out of order). +# * MySQL - CH server accesses itself via mysql table function. +# * PostgreSQL - CH server accesses itself via postgresql table function, but can't execute query (No LOGIN SUCCESS entry). # * gRPC - not done yet # # There is way to control how many time a query (e.g. via mysql table function) is retried @@ -53,7 +51,7 @@ function reportError() function executeQuery() { - ## Execute query (provided via heredoc or herestring) and print query in case of error. + # Execute query (provided via heredoc or herestring) and print query in case of error. trap 'rm -f ${TMP_QUERY_FILE}; trap - ERR RETURN' RETURN # Since we want to report with current values supplied to this function call # shellcheck disable=SC2064 @@ -82,7 +80,7 @@ trap "cleanup" EXIT function executeQueryExpectError() { cat - > "${TMP_QUERY_FILE}" - ! ${CLICKHOUSE_CLIENT} "${@}" --multiquery --queries-file "${TMP_QUERY_FILE}" 2>&1 | tee -a ${TMP_QUERY_FILE} + ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a ${TMP_QUERY_FILE} } function createUser() @@ -121,6 +119,8 @@ function createUser() executeQuery < Date: Fri, 28 Jul 2023 07:08:11 +0000 Subject: [PATCH 64/72] use same executor for GET_PART and ATTACH_PART --- 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 d62a1d960e6..2c2cea0af2b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3311,7 +3311,7 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne auto job_type = selected_entry->log_entry->type; /// Depending on entry type execute in fetches (small) pool or big merge_mutate pool - if (job_type == LogEntry::GET_PART) + if (job_type == LogEntry::GET_PART || job_type == LogEntry::ATTACH_PART) { assignee.scheduleFetchTask(std::make_shared( [this, selected_entry] () mutable From 63b05da1f2da6cee086d1154ddc670329aba667d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 28 Jul 2023 07:23:34 +0000 Subject: [PATCH 65/72] System logs improvements --- .../settings.md | 128 ++++++++++- docs/en/operations/system-tables/index.md | 4 + .../settings.md | 212 +++++++++++++++--- docs/ru/operations/system-tables/index.md | 4 + programs/server/config.xml | 53 +++++ src/Common/SystemLogBase.cpp | 81 ++++--- src/Common/SystemLogBase.h | 43 +++- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Daemon/BaseDaemon.cpp | 4 + .../IO/AsynchronousBoundedReadBuffer.cpp | 2 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- src/Interpreters/Context.cpp | 6 + src/Interpreters/Context.h | 3 + src/Interpreters/CrashLog.cpp | 5 +- src/Interpreters/CrashLog.h | 5 + src/Interpreters/MetricLog.cpp | 2 +- src/Interpreters/PartLog.cpp | 2 +- src/Interpreters/ProcessorsProfileLog.cpp | 7 - src/Interpreters/ProcessorsProfileLog.h | 7 +- src/Interpreters/Session.cpp | 2 +- src/Interpreters/SessionLog.cpp | 6 +- src/Interpreters/SystemLog.cpp | 92 +++++--- src/Interpreters/SystemLog.h | 17 +- src/Interpreters/TextLog.cpp | 11 +- src/Interpreters/TextLog.h | 13 +- src/Interpreters/ThreadStatusExt.cpp | 4 +- src/Interpreters/TraceCollector.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 2 +- src/Interpreters/TransactionsInfoLog.cpp | 2 +- src/Loggers/Loggers.cpp | 44 +++- src/Loggers/OwnSplitChannel.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- tests/integration/parallel_skip.json | 9 +- .../test_crash_log/configs/crash_log.xml | 16 ++ tests/integration/test_crash_log/test.py | 19 +- .../test_system_flush_logs/test.py | 99 +++++++- .../test_system_logs/test_system_logs.py | 50 +++++ 40 files changed, 794 insertions(+), 178 deletions(-) create mode 100644 tests/integration/test_crash_log/configs/crash_log.xml diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a6ae517e401..e9f0f0dae00 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -512,7 +512,7 @@ Both the cache for `local_disk`, and temporary data will be stored in `/tiny_loc cache local_disk /tiny_local_cache/ - 10M + 10M 1M 1 0 @@ -1592,6 +1592,10 @@ To manually turn on metrics history collection [`system.metric_log`](../../opera metric_log
7500 1000 + 1048576 + 8192 + 524288 + false ``` @@ -1695,6 +1699,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1706,6 +1718,10 @@ Use the following parameters to configure logging: part_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1773,6 +1789,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1786,6 +1810,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_log
Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1831,6 +1859,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size_rows, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1844,6 +1880,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_thread_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1861,6 +1901,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1874,6 +1922,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_views_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1890,6 +1942,14 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1901,13 +1961,16 @@ Parameters: system text_log
7500 + 1048576 + 8192 + 524288 + false Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day ``` - ## trace_log {#server_configuration_parameters-trace_log} Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. @@ -1920,6 +1983,12 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1931,6 +2000,10 @@ The default server configuration file `config.xml` contains the following settin trace_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1945,9 +2018,18 @@ Parameters: - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) **Example** + ```xml @@ -1955,11 +2037,53 @@ Parameters: asynchronous_insert_log
7500 toYYYYMM(event_date) + 1048576 + 8192 + 524288 + false
``` +## crash_log {#server_configuration_parameters-crash_log} + +Settings for the [crash_log](../../operations/system-tables/crash-log.md) system table operation. + +Parameters: + +- `database` — Database for storing a table. +- `table` — Table name. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. +- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). + +The default server configuration file `config.xml` contains the following settings section: + +``` xml + + system + crash_log
+ toYYYYMM(event_date) + 7500 + 1024 + 1024 + 512 + false +
+``` + ## query_masking_rules {#query-masking-rules} Regexp-based rules, which will be applied to queries as well as all log messages before storing them in server logs, diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 1b720098fc7..a46f306f677 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -47,6 +47,10 @@ An example: ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 --> 7500 + 1048576 + 8192 + 524288 + false ``` diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 421df3fe3eb..81a696bcfc1 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1058,6 +1058,10 @@ ClickHouse использует потоки из глобального пул metric_log
7500 1000 + 1048576 + 8192 + 524288 + false ``` @@ -1155,12 +1159,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. - +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. **Пример** ``` xml @@ -1169,6 +1180,10 @@ ClickHouse использует потоки из глобального пул part_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1218,11 +1233,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы, куда будет записываться лог; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1234,6 +1257,10 @@ ClickHouse использует потоки из глобального пул query_log
Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1245,11 +1272,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы, куда будет записываться лог; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1261,6 +1296,10 @@ ClickHouse использует потоки из глобального пул query_thread_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1272,11 +1311,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` – имя базы данных. -- `table` – имя системной таблицы, где будут логироваться запросы. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать, если задан параметр `engine`. -- `engine` — устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать, если задан параметр `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1288,6 +1335,10 @@ ClickHouse использует потоки из глобального пул query_views_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1297,12 +1348,20 @@ ClickHouse использует потоки из глобального пул Параметры: -- `level` — Максимальный уровень сообщения (по умолчанию `Trace`) которое будет сохранено в таблице. -- `database` — имя базы данных для хранения таблицы. -- `table` — имя таблицы, куда будут записываться текстовые сообщения. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `level` — Максимальный уровень сообщения (по умолчанию `Trace`) которое будет сохранено в таблице. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. **Пример** ```xml @@ -1312,6 +1371,10 @@ ClickHouse использует потоки из глобального пул system text_log
7500 + 1048576 + 8192 + 524288 + false Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day @@ -1323,13 +1386,21 @@ ClickHouse использует потоки из глобального пул Настройки для [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. -Parameters: +Параметры: -- `database` — Database for storing a table. -- `table` — Table name. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. По умолчанию файл настроек сервера `config.xml` содержит следующие настройки: @@ -1339,9 +1410,84 @@ Parameters: trace_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 ``` +## asynchronous_insert_log {#server_configuration_parameters-asynchronous_insert_log} + +Настройки для asynchronous_insert_log Система для логирования ассинхронных вставок. + +Параметры: + +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. + +**Пример** + +```xml + + + system + asynchronous_insert_log
+ 7500 + toYYYYMM(event_date) + 1048576 + 8192 + 524288 + +
+
+``` + +## crash_log {#server_configuration_parameters-crash_log} + +Настройки для таблицы [crash_log](../../operations/system-tables/crash-log.md). + +Параметры: + +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1024. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 1024. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: true. + +**Пример** + +``` xml + + system + crash_log
+ toYYYYMM(event_date) + 7500 + 1024 + 1024 + 512 + true +
+``` + ## query_masking_rules {#query-masking-rules} Правила, основанные на регулярных выражениях, которые будут применены для всех запросов, а также для всех сообщений перед сохранением их в лог на сервере, diff --git a/docs/ru/operations/system-tables/index.md b/docs/ru/operations/system-tables/index.md index 7ff368b1910..24f79cae212 100644 --- a/docs/ru/operations/system-tables/index.md +++ b/docs/ru/operations/system-tables/index.md @@ -45,6 +45,10 @@ sidebar_label: "Системные таблицы" ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 --> 7500 + 1048576 + 8192 + 524288 + false ``` diff --git a/programs/server/config.xml b/programs/server/config.xml index 2a7dc1e576a..153cb728bb4 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1026,6 +1026,14 @@ 7500 + + 1048576 + + 8192 + + 524288 + + false @@ -1039,6 +1047,11 @@ toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + + false @@ -1084,7 +1109,11 @@ system metric_log
7500 + 1048576 + 8192 + 524288 1000 + false @@ -1151,6 +1196,10 @@ toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false