From 07582d56f32d72a3f13d9e7303310ff1753c97f5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 13:18:13 +0000 Subject: [PATCH 01/79] Add config for pinning api version --- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperConstants.h | 4 +- src/Coordination/KeeperContext.cpp | 37 ++++++++ src/Coordination/KeeperContext.h | 6 ++ src/Coordination/KeeperServer.cpp | 2 + src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- .../__init__.py | 0 .../configs/enable_keeper.xml | 31 +++++++ .../test_keeper_api_version_config/test.py | 87 +++++++++++++++++++ 11 files changed, 169 insertions(+), 6 deletions(-) create mode 100644 src/Coordination/KeeperContext.cpp create mode 100644 tests/integration/test_keeper_api_version_config/__init__.py create mode 100644 tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_api_version_config/test.py diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 7077e792fd8..a64969e3d31 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -486,7 +486,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(static_cast(Coordination::current_keeper_api_version)); + return toString(static_cast(Coordination::latest_keeper_api_version)); } String CreateSnapshotCommand::run() diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 4b5a5b54be0..42161eee908 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -13,7 +13,7 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -inline constexpr auto current_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; +inline constexpr auto latest_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; @@ -21,7 +21,7 @@ const std::string keeper_api_version_path = keeper_system_path + "/api_version"; using PathWithData = std::pair; const std::vector child_system_paths_with_data { - {keeper_api_version_path, toString(static_cast(current_keeper_api_version))} + {keeper_api_version_path, toString(static_cast(latest_keeper_api_version))} }; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp new file mode 100644 index 00000000000..d420242e670 --- /dev/null +++ b/src/Coordination/KeeperContext.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +DECLARE_SETTING_ENUM(KeeperApiVersion); +IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, + {{"ZOOKEEPER_COMPATIBLE", KeeperApiVersion::ZOOKEEPER_COMPATIBLE}, + {"WITH_FILTERED_LIST", KeeperApiVersion::WITH_FILTERED_LIST}, + {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, + {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ + for (const auto & [path, data] : child_system_paths_with_data) + system_nodes_with_data[std::string{path}] = data; + + if (config.has("keeper_server.api_version")) + { + auto version_string = config.getString("keeper_server.api_version"); + auto api_version = SettingFieldKeeperApiVersionTraits::fromString(version_string); + LOG_INFO(&Poco::Logger::get("KeeperContext"), "API version override used: {}", version_string); + system_nodes_with_data[keeper_api_version_path] = toString(static_cast(api_version)); + } +} + +} diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 64fa8cea6ec..f26009c9af0 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,14 @@ #pragma once +#include + namespace DB { struct KeeperContext { + void initialize(const Poco::Util::AbstractConfiguration & config); + enum class Phase : uint8_t { INIT, @@ -16,6 +20,8 @@ struct KeeperContext bool ignore_system_path_on_startup{false}; bool digest_enabled{true}; + + std::unordered_map system_nodes_with_data; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 45db9e85fa5..897d7e05671 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -119,6 +119,8 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); + keeper_context->initialize(config); + keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 8b80db3e520..a2d9d8136cd 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -185,7 +185,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } /// Serialize data tree - writeBinary(snapshot.snapshot_container_size - child_system_paths_with_data.size(), out); + writeBinary(snapshot.snapshot_container_size - keeper_context->system_nodes_with_data.size(), out); size_t counter = 0; for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++counter) { diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 66d6b0f5843..8abcd062b7f 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -283,7 +283,7 @@ void KeeperStorage::initializeSystemNodes() } // insert child system nodes - for (const auto & [path, data] : child_system_paths_with_data) + for (const auto & [path, data] : keeper_context->system_nodes_with_data) { assert(keeper_api_version_path.starts_with(keeper_system_path)); Node child_system_node; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 453fd0f2e60..005c67ad261 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2357,7 +2357,7 @@ TEST_P(CoordinationTest, TestCurrentApiVersion) uint8_t keeper_version{0}; DB::ReadBufferFromOwnString buf(get_response.data); DB::readIntText(keeper_version, buf); - EXPECT_EQ(keeper_version, static_cast(current_keeper_api_version)); + EXPECT_EQ(keeper_version, static_cast(latest_keeper_api_version)); } TEST_P(CoordinationTest, TestSystemNodeModify) diff --git a/tests/integration/test_keeper_api_version_config/__init__.py b/tests/integration/test_keeper_api_version_config/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml b/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml new file mode 100644 index 00000000000..c153d025598 --- /dev/null +++ b/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml @@ -0,0 +1,31 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 10 + 5 + 5000 + 10000 + trace + + + 0 + 0 + 0 + + + + + + + 1 + localhost + 9234 + + + + diff --git a/tests/integration/test_keeper_api_version_config/test.py b/tests/integration/test_keeper_api_version_config/test.py new file mode 100644 index 00000000000..34d3acc4f04 --- /dev/null +++ b/tests/integration/test_keeper_api_version_config/test.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python3 + +import pytest +import os +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient, KazooState + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(api_version=None, expect_fail=True): + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + + if api_version: + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + f"{api_version}<\\/api_version>", + ) + + node.start_clickhouse(retry_start=not expect_fail) + keeper_utils.wait_until_connected(cluster, node) + + +def test_keeper_api_version(started_cluster): + restart_clickhouse() + + def assert_version(string_version, version_number): + node.wait_for_log_line( + f"Detected server's API version: {string_version}", look_behind_lines=1000 + ) + + try: + node_zk = get_connection_zk(node.name) + assert node_zk.get("/keeper/api_version")[0] == str(version_number).encode() + finally: + if node_zk: + node_zk.stop() + node_zk.close() + + assert_version("WITH_CHECK_NOT_EXISTS", 3) + + for i, version in enumerate( + [ + "ZOOKEEPER_COMPATIBLE", + "WITH_FILTERED_LIST", + "WITH_MULTI_READ", + "WITH_CHECK_NOT_EXISTS", + ] + ): + restart_clickhouse(version) + assert_version(version, i) + + with pytest.raises(Exception): + restart_clickhouse("INVALID_VERSION", expect_fail=True) From b8b70d78e15948dd278d298d42bc1e4ef2d4fc0d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 13:34:27 +0000 Subject: [PATCH 02/79] Build fix --- contrib/azure-cmake/CMakeLists.txt | 2 +- programs/keeper/CMakeLists.txt | 1 + src/Coordination/KeeperContext.cpp | 7 +++++-- src/Coordination/KeeperContext.h | 2 ++ 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 9c361db47ca..23e38e6b63d 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR ARCH_PPC64LE) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR OS_FREEBSD OR ARCH_PPC64LE) message(STATUS "Not using Azure blob storage") return() endif() diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 1f1138f49eb..4f74cc06801 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -43,6 +43,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperDispatcher.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperLogStore.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperServer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index d420242e670..a750f2e1860 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -20,11 +20,14 @@ IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +KeeperContext::KeeperContext() { for (const auto & [path, data] : child_system_paths_with_data) system_nodes_with_data[std::string{path}] = data; - +} + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ if (config.has("keeper_server.api_version")) { auto version_string = config.getString("keeper_server.api_version"); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index f26009c9af0..de502b6c566 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -7,6 +7,8 @@ namespace DB struct KeeperContext { + KeeperContext(); + void initialize(const Poco::Util::AbstractConfiguration & config); enum class Phase : uint8_t From b634012feb40445079145639d23c44967c00547e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 10:57:03 +0000 Subject: [PATCH 03/79] Add feature flags for Keeper API --- src/Common/ZooKeeper/IKeeper.h | 4 +- src/Common/ZooKeeper/TestKeeper.h | 5 +- src/Common/ZooKeeper/ZooKeeper.cpp | 4 +- src/Common/ZooKeeper/ZooKeeper.h | 7 +- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 1 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 60 +++++++---- src/Common/ZooKeeper/ZooKeeperImpl.h | 7 +- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 4 +- src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/FourLetterCommand.cpp | 27 ++++- src/Coordination/FourLetterCommand.h | 12 +++ src/Coordination/KeeperConstants.h | 9 +- src/Coordination/KeeperContext.cpp | 46 ++++---- src/Coordination/KeeperContext.h | 3 + src/Coordination/KeeperDispatcher.cpp | 47 ++++---- src/Coordination/KeeperDispatcher.h | 8 ++ src/Coordination/KeeperFeatureFlags.cpp | 102 ++++++++++++++++++ src/Coordination/KeeperFeatureFlags.h | 49 +++++++++ src/Coordination/KeeperServer.cpp | 8 +- src/Coordination/KeeperServer.h | 1 + src/Coordination/KeeperStorage.cpp | 4 +- src/Coordination/tests/gtest_coordination.cpp | 15 +-- .../StorageSystemZooKeeperConnection.cpp | 4 +- .../test_keeper_api_version_config/test.py | 87 --------------- .../__init__.py | 0 .../configs/enable_keeper.xml | 2 +- .../test_keeper_feature_flags_config/test.py | 92 ++++++++++++++++ 27 files changed, 422 insertions(+), 188 deletions(-) create mode 100644 src/Coordination/KeeperFeatureFlags.cpp create mode 100644 src/Coordination/KeeperFeatureFlags.h delete mode 100644 tests/integration/test_keeper_api_version_config/test.py rename tests/integration/{test_keeper_api_version_config => test_keeper_feature_flags_config}/__init__.py (100%) rename tests/integration/{test_keeper_api_version_config => test_keeper_feature_flags_config}/configs/enable_keeper.xml (97%) create mode 100644 tests/integration/test_keeper_feature_flags_config/test.py diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 3eb5819df90..369aacf16c7 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -530,7 +530,7 @@ public: const Requests & requests, MultiCallback callback) = 0; - virtual DB::KeeperApiVersion getApiVersion() const = 0; + virtual bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const = 0; /// Expire session and finish all pending requests virtual void finalize(const String & reason) = 0; diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 4bffa4e1d4f..9bbd018cfb1 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace Coordination @@ -92,9 +93,9 @@ public: void finalize(const String & reason) override; - DB::KeeperApiVersion getApiVersion() const override + bool isFeatureEnabled(DB::KeeperFeatureFlag) const override { - return KeeperApiVersion::ZOOKEEPER_COMPATIBLE; + return false; } struct Node diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a587ad6caf4..826032fc56b 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -865,9 +865,9 @@ bool ZooKeeper::expired() return impl->isExpired(); } -DB::KeeperApiVersion ZooKeeper::getApiVersion() const +bool ZooKeeper::isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const { - return impl->getApiVersion(); + return impl->isFeatureEnabled(feature_flag); } Int64 ZooKeeper::getClientID() diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 96f9914b597..a4a631b9d5a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -215,7 +216,7 @@ public: /// Returns true, if the session has expired. bool expired(); - DB::KeeperApiVersion getApiVersion() const; + bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const; /// Create a znode. /// Throw an exception if something went wrong. @@ -553,7 +554,7 @@ private: template MultiReadResponses multiRead(TIter start, TIter end, RequestFactory request_factory, AsyncFunction async_fun) { - if (getApiVersion() >= DB::KeeperApiVersion::WITH_MULTI_READ) + if (isFeatureEnabled(DB::KeeperFeatureFlag::MULTI_READ)) { Coordination::Requests requests; for (auto it = start; it != end; ++it) @@ -685,7 +686,7 @@ String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config); template void addCheckNotExistsRequest(Coordination::Requests & requests, const Client & client, const std::string & path) { - if (client.getApiVersion() >= DB::KeeperApiVersion::WITH_CHECK_NOT_EXISTS) + if (client.isFeatureEnabled(DB::KeeperFeatureFlag::CHECK_NOT_EXISTS)) { auto request = std::make_shared(); request->path = path; diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 0ebc7f667cb..e99285b0056 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7f23ac00efe..bf0d1871244 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -354,7 +354,8 @@ ZooKeeper::ZooKeeper( send_thread = ThreadFromGlobalPool([this] { sendThread(); }); receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); - initApiVersion(); + initFeatureFlags(); + keeper_feature_flags.logFlags(log); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); } @@ -1089,41 +1090,58 @@ void ZooKeeper::pushRequest(RequestInfo && info) ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); } -KeeperApiVersion ZooKeeper::getApiVersion() const +bool ZooKeeper::isFeatureEnabled(KeeperFeatureFlag feature_flag) const { - return keeper_api_version; + return keeper_feature_flags.isEnabled(feature_flag); } -void ZooKeeper::initApiVersion() +void ZooKeeper::initFeatureFlags() { - auto promise = std::make_shared>(); - auto future = promise->get_future(); - - auto callback = [promise](const Coordination::GetResponse & response) mutable + const auto try_get = [&](const std::string & path, const std::string & description) -> std::optional { - promise->set_value(response); + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise](const Coordination::GetResponse & response) mutable + { + promise->set_value(response); + }; + + get(path, std::move(callback), {}); + if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + { + LOG_TRACE(log, "Failed to get {}: timeout", description); + return std::nullopt; + } + + auto response = future.get(); + + if (response.error != Coordination::Error::ZOK) + { + LOG_TRACE(log, "Failed to get {}", description); + return std::nullopt; + } + + return std::move(response.data); }; - get(keeper_api_version_path, std::move(callback), {}); - if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + if (auto feature_flags = try_get(keeper_api_feature_flags_path, "feature flags"); feature_flags.has_value()) { - LOG_TRACE(log, "Failed to get API version: timeout"); + keeper_feature_flags.setFeatureFlags(std::move(*feature_flags)); return; } - auto response = future.get(); + auto keeper_api_version_string = try_get(keeper_api_version_path, "API version"); - if (response.error != Coordination::Error::ZOK) - { - LOG_TRACE(log, "Failed to get API version"); + if (!keeper_api_version_string.has_value()) return; - } + DB::ReadBufferFromOwnString buf(*keeper_api_version_string); uint8_t keeper_version{0}; - DB::ReadBufferFromOwnString buf(response.data); DB::readIntText(keeper_version, buf); - keeper_api_version = static_cast(keeper_version); + auto keeper_api_version = static_cast(keeper_version); LOG_TRACE(log, "Detected server's API version: {}", keeper_api_version); + keeper_feature_flags.fromApiVersion(keeper_api_version); } @@ -1243,7 +1261,7 @@ void ZooKeeper::list( WatchCallback watch) { std::shared_ptr request{nullptr}; - if (keeper_api_version < Coordination::KeeperApiVersion::WITH_FILTERED_LIST) + if (!isFeatureEnabled(KeeperFeatureFlag::FILTERED_LIST)) { if (list_request_type != ListRequestType::ALL) throw Exception(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); @@ -1308,7 +1326,7 @@ void ZooKeeper::multi( { ZooKeeperMultiRequest request(requests, default_acls); - if (request.getOpNum() == OpNum::MultiRead && keeper_api_version < Coordination::KeeperApiVersion::WITH_MULTI_READ) + if (request.getOpNum() == OpNum::MultiRead && !isFeatureEnabled(KeeperFeatureFlag::MULTI_READ)) throw Exception(Error::ZBADARGUMENTS, "MultiRead request type cannot be used because it's not supported by the server"); RequestInfo request_info; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 944c5032fac..ae6bef067e3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -181,7 +182,7 @@ public: const Requests & requests, MultiCallback callback) override; - DB::KeeperApiVersion getApiVersion() const override; + bool isFeatureEnabled(KeeperFeatureFlag feature_flag) const override; /// Without forcefully invalidating (finalizing) ZooKeeper session before /// establishing a new one, there was a possibility that server is using @@ -312,12 +313,12 @@ private: void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_ms = 0); - void initApiVersion(); + void initFeatureFlags(); CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession}; std::shared_ptr zk_log; - DB::KeeperApiVersion keeper_api_version{DB::KeeperApiVersion::ZOOKEEPER_COMPATIBLE}; + DB::KeeperFeatureFlags keeper_feature_flags; }; } diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index bf99cb76798..9d02d674010 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -402,9 +402,9 @@ public: ephemeral_nodes.clear(); } - KeeperApiVersion getApiVersion() const + bool isFeatureEnabled(KeeperFeatureFlag feature_flag) const { - return keeper->getApiVersion(); + return keeper->isFeatureEnabled(feature_flag); } private: diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 7a66134f43f..edbdfd4f9fc 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } -const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs"; +const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs,ftfl"; KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() : server_id(NOT_EXIST) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index a64969e3d31..55120e70d99 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Coordination/KeeperFeatureFlags.h" #include #include #include @@ -153,6 +154,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat FourLetterCommandPtr clean_resources_command = std::make_shared(keeper_dispatcher); factory.registerCommand(clean_resources_command); + FourLetterCommandPtr feature_flags_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(feature_flags_command); + factory.initializeAllowList(keeper_dispatcher); factory.setInitialize(true); } @@ -486,7 +490,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(static_cast(Coordination::latest_keeper_api_version)); + return toString(0); } String CreateSnapshotCommand::run() @@ -535,4 +539,25 @@ String CleanResourcesCommand::run() return "ok"; } +String FeatureFlagsCommand::run() +{ + const auto & feature_flags = keeper_dispatcher.getKeeperContext()->feature_flags; + + StringBuffer ret; + + auto append = [&ret] (String key, uint8_t value) -> void + { + writeText(key, ret); + writeText('\t', ret); + writeText(std::to_string(value), ret); + writeText('\n', ret); + }; + + for (const auto feature : all_keeper_feature_flags) + append(SettingFieldKeeperFeatureFlagTraits::toString(feature), feature_flags.isEnabled(feature)); + + return ret.str(); + +} + } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index c1a91303c05..eb2cf9419ae 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -401,4 +401,16 @@ struct CleanResourcesCommand : public IFourLetterCommand ~CleanResourcesCommand() override = default; }; +struct FeatureFlagsCommand : public IFourLetterCommand +{ + explicit FeatureFlagsCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } + + String name() override { return "ftfl"; } + String run() override; + ~FeatureFlagsCommand() override = default; +}; + } diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 42161eee908..6d9c03ca8a9 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -5,6 +5,7 @@ namespace DB { +/// left for backwards compatibility enum class KeeperApiVersion : uint8_t { ZOOKEEPER_COMPATIBLE = 0, @@ -13,15 +14,9 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -inline constexpr auto latest_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; - -using PathWithData = std::pair; -const std::vector child_system_paths_with_data -{ - {keeper_api_version_path, toString(static_cast(latest_keeper_api_version))} -}; +const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index a750f2e1860..dc408afc19a 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -1,40 +1,42 @@ #include #include -#include -#include #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -DECLARE_SETTING_ENUM(KeeperApiVersion); -IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, - {{"ZOOKEEPER_COMPATIBLE", KeeperApiVersion::ZOOKEEPER_COMPATIBLE}, - {"WITH_FILTERED_LIST", KeeperApiVersion::WITH_FILTERED_LIST}, - {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, - {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); - KeeperContext::KeeperContext() { - for (const auto & [path, data] : child_system_paths_with_data) - system_nodes_with_data[std::string{path}] = data; + /// enable by default some feature flags + feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); + feature_flags.enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); + system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); } void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) { - if (config.has("keeper_server.api_version")) + digest_enabled = config.getBool("keeper_server.digest_enabled", false); + ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + + static const std::string feature_flags_key = "keeper_server.feature_flags"; + if (config.has(feature_flags_key)) { - auto version_string = config.getString("keeper_server.api_version"); - auto api_version = SettingFieldKeeperApiVersionTraits::fromString(version_string); - LOG_INFO(&Poco::Logger::get("KeeperContext"), "API version override used: {}", version_string); - system_nodes_with_data[keeper_api_version_path] = toString(static_cast(api_version)); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(feature_flags_key, keys); + for (const auto & key : keys) + { + auto feature_flag = SettingFieldKeeperFeatureFlagTraits::fromString(key); + auto is_enabled = config.getBool(feature_flags_key + "." + key); + if (is_enabled) + feature_flags.enableFeatureFlag(feature_flag); + else + feature_flags.disableFeatureFlag(feature_flag); + } + + system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); } + + feature_flags.logFlags(&Poco::Logger::get("KeeperContext")); } } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index de502b6c566..9b7000fa726 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -24,6 +25,8 @@ struct KeeperContext bool digest_enabled{true}; std::unordered_map system_nodes_with_data; + + KeeperFeatureFlags feature_flags; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index d64134f3024..7318a492b35 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -336,28 +336,39 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) - { - /// check if we have queue of read requests depending on this request to be committed - std::lock_guard lock(read_request_queue_mutex); - if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) + keeper_context = std::make_shared(); + keeper_context->initialize(config); + + server = std::make_unique( + configuration_and_settings, + config, + responses_queue, + snapshots_queue, + keeper_context, + snapshot_s3, + [this](const KeeperStorage::RequestForSession & request_for_session) { - auto & xid_to_request_queue = it->second; - - if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) + /// check if we have queue of read requests depending on this request to be committed + std::lock_guard lock(read_request_queue_mutex); + if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - for (const auto & read_request : request_queue_it->second) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - } + auto & xid_to_request_queue = it->second; - xid_to_request_queue.erase(request_queue_it); + if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); + request_queue_it != xid_to_request_queue.end()) + { + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + xid_to_request_queue.erase(request_queue_it); + } } - } - }); + }); try { diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 4b8b134cf8f..1759f55d981 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -81,6 +81,8 @@ private: KeeperSnapshotManagerS3 snapshot_s3; + KeeperContextPtr keeper_context; + /// Thread put requests to raft void requestThread(); /// Thread put responses for subscribed sessions @@ -198,6 +200,12 @@ public: return configuration_and_settings; } + const KeeperContextPtr & getKeeperContext() const + { + return keeper_context; + } + + void incrementPacketsSent() { keeper_stats.incrementPacketsSent(); diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp new file mode 100644 index 00000000000..216dca014d4 --- /dev/null +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +std::pair getByteAndBitIndex(size_t num) +{ + size_t byte_idx = num / 8; + auto bit_idx = (7 - num % 8); + return {byte_idx, bit_idx}; +} + +} + +IMPLEMENT_SETTING_ENUM(KeeperFeatureFlag, ErrorCodes::BAD_ARGUMENTS, + {{"filtered_list", KeeperFeatureFlag::FILTERED_LIST}, + {"multi_read", KeeperFeatureFlag::MULTI_READ}, + {"check_not_exists", KeeperFeatureFlag::CHECK_NOT_EXISTS}}); + +KeeperFeatureFlags::KeeperFeatureFlags() +{ + /// get byte idx of largest value + auto [byte_idx, _] = getByteAndBitIndex(all_keeper_feature_flags.size() - 1); + feature_flags = std::string(byte_idx + 1, 0); +} + +KeeperFeatureFlags::KeeperFeatureFlags(std::string feature_flags_) + : feature_flags(std::move(feature_flags_)) +{} + +void KeeperFeatureFlags::fromApiVersion(KeeperApiVersion keeper_api_version) +{ + if (keeper_api_version == KeeperApiVersion::ZOOKEEPER_COMPATIBLE) + return; + + if (keeper_api_version >= KeeperApiVersion::WITH_FILTERED_LIST) + enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); + + if (keeper_api_version >= KeeperApiVersion::WITH_MULTI_READ) + enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); + + if (keeper_api_version >= KeeperApiVersion::WITH_CHECK_NOT_EXISTS) + enableFeatureFlag(KeeperFeatureFlag::CHECK_NOT_EXISTS); +} + +bool KeeperFeatureFlags::isEnabled(KeeperFeatureFlag feature_flag) const +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + + if (byte_idx > feature_flags.size()) + return false; + + return feature_flags[byte_idx] & (1 << bit_idx); +} + +void KeeperFeatureFlags::setFeatureFlags(std::string feature_flags_) +{ + feature_flags = std::move(feature_flags_); +} + +void KeeperFeatureFlags::enableFeatureFlag(KeeperFeatureFlag feature_flag) +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + chassert(byte_idx < feature_flags.size()); + + feature_flags[byte_idx] |= (1 << bit_idx); +} + +void KeeperFeatureFlags::disableFeatureFlag(KeeperFeatureFlag feature_flag) +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + chassert(byte_idx < feature_flags.size()); + + feature_flags[byte_idx] &= ~(1 << bit_idx); +} + +const std::string & KeeperFeatureFlags::getFeatureFlags() const +{ + return feature_flags; +} + +void KeeperFeatureFlags::logFlags(Poco::Logger * log) const +{ + for (const auto & feature_flag : all_keeper_feature_flags) + { + auto is_enabled = isEnabled(feature_flag); + LOG_INFO(log, "Keeper feature flag {}: {}", SettingFieldKeeperFeatureFlagTraits::toString(feature_flag), is_enabled ? "enabled" : "disabled"); + } +} + +} diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Coordination/KeeperFeatureFlags.h new file mode 100644 index 00000000000..cdd4704a7ca --- /dev/null +++ b/src/Coordination/KeeperFeatureFlags.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +enum KeeperFeatureFlag +{ + FILTERED_LIST = 0, + MULTI_READ, + CHECK_NOT_EXISTS, +}; + +static inline constexpr std::array all_keeper_feature_flags +{ + KeeperFeatureFlag::FILTERED_LIST, + KeeperFeatureFlag::MULTI_READ, + KeeperFeatureFlag::CHECK_NOT_EXISTS, +}; + +DECLARE_SETTING_ENUM(KeeperFeatureFlag); + +class KeeperFeatureFlags +{ +public: + KeeperFeatureFlags(); + + explicit KeeperFeatureFlags(std::string feature_flags_); + + /// backwards compatibility + void fromApiVersion(KeeperApiVersion keeper_api_version); + + bool isEnabled(KeeperFeatureFlag feature) const; + + void setFeatureFlags(std::string feature_flags_); + const std::string & getFeatureFlags() const; + + void enableFeatureFlag(KeeperFeatureFlag feature); + void disableFeatureFlag(KeeperFeatureFlag feature); + + void logFlags(Poco::Logger * log) const; +private: + std::string feature_flags; +}; + +} diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b3150f11cfb..43719e5a69e 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -108,23 +108,19 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared()} + , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - keeper_context->initialize(config); - - keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); - keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); - state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index db4e9c1962e..8f416b1f48c 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,6 +72,7 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 8abcd062b7f..ecf795f4068 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -285,7 +285,7 @@ void KeeperStorage::initializeSystemNodes() // insert child system nodes for (const auto & [path, data] : keeper_context->system_nodes_with_data) { - assert(keeper_api_version_path.starts_with(keeper_system_path)); + assert(path.starts_with(keeper_system_path)); Node child_system_node; child_system_node.setData(data); auto [map_key, _] = container.insert(std::string{path}, child_system_node); @@ -1060,7 +1060,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce ProfileEvents::increment(ProfileEvents::KeeperGetRequest); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); - if (request.path == Coordination::keeper_api_version_path) + if (request.path == Coordination::keeper_api_feature_flags_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 005c67ad261..2793b23c572 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2,7 +2,9 @@ #include #include "Common/ZooKeeper/IKeeper.h" +#include "Coordination/KeeperConstants.h" #include "Coordination/KeeperContext.h" +#include "Coordination/KeeperFeatureFlags.h" #include "Coordination/KeeperStorage.h" #include "Core/Defines.h" #include "IO/WriteHelpers.h" @@ -2346,18 +2348,19 @@ TEST_P(CoordinationTest, TestDurableState) } } -TEST_P(CoordinationTest, TestCurrentApiVersion) +TEST_P(CoordinationTest, TestFeatureFlags) { using namespace Coordination; KeeperStorage storage{500, "", keeper_context}; auto request = std::make_shared(); - request->path = DB::keeper_api_version_path; + request->path = DB::keeper_api_feature_flags_path; auto responses = storage.processRequest(request, 0, std::nullopt, true, true); const auto & get_response = getSingleResponse(responses); - uint8_t keeper_version{0}; - DB::ReadBufferFromOwnString buf(get_response.data); - DB::readIntText(keeper_version, buf); - EXPECT_EQ(keeper_version, static_cast(latest_keeper_api_version)); + DB::KeeperFeatureFlags feature_flags; + feature_flags.setFeatureFlags(get_response.data); + ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST)); + ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ)); + ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS)); } TEST_P(CoordinationTest, TestSystemNodeModify) diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 559e12ad5ee..9a6a592f2c4 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(context->getZooKeeper()->getApiVersion()); + res_columns[6]->insert(0); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(elem.second->getApiVersion()); + res_columns[6]->insert(0); res_columns[7]->insert(elem.second->getClientID()); } diff --git a/tests/integration/test_keeper_api_version_config/test.py b/tests/integration/test_keeper_api_version_config/test.py deleted file mode 100644 index 34d3acc4f04..00000000000 --- a/tests/integration/test_keeper_api_version_config/test.py +++ /dev/null @@ -1,87 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -import os -from helpers.cluster import ClickHouseCluster -import helpers.keeper_utils as keeper_utils -from kazoo.client import KazooClient, KazooState - -CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) -cluster = ClickHouseCluster(__file__) - -# clickhouse itself will use external zookeeper -node = cluster.add_instance( - "node", - main_configs=["configs/enable_keeper.xml"], - stay_alive=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def get_connection_zk(nodename, timeout=30.0): - _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout - ) - _fake_zk_instance.start() - return _fake_zk_instance - - -def restart_clickhouse(api_version=None, expect_fail=True): - node.stop_clickhouse() - node.copy_file_to_container( - os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), - "/etc/clickhouse-server/config.d/enable_keeper.xml", - ) - - if api_version: - node.replace_in_config( - "/etc/clickhouse-server/config.d/enable_keeper.xml", - "", - f"{api_version}<\\/api_version>", - ) - - node.start_clickhouse(retry_start=not expect_fail) - keeper_utils.wait_until_connected(cluster, node) - - -def test_keeper_api_version(started_cluster): - restart_clickhouse() - - def assert_version(string_version, version_number): - node.wait_for_log_line( - f"Detected server's API version: {string_version}", look_behind_lines=1000 - ) - - try: - node_zk = get_connection_zk(node.name) - assert node_zk.get("/keeper/api_version")[0] == str(version_number).encode() - finally: - if node_zk: - node_zk.stop() - node_zk.close() - - assert_version("WITH_CHECK_NOT_EXISTS", 3) - - for i, version in enumerate( - [ - "ZOOKEEPER_COMPATIBLE", - "WITH_FILTERED_LIST", - "WITH_MULTI_READ", - "WITH_CHECK_NOT_EXISTS", - ] - ): - restart_clickhouse(version) - assert_version(version, i) - - with pytest.raises(Exception): - restart_clickhouse("INVALID_VERSION", expect_fail=True) diff --git a/tests/integration/test_keeper_api_version_config/__init__.py b/tests/integration/test_keeper_feature_flags_config/__init__.py similarity index 100% rename from tests/integration/test_keeper_api_version_config/__init__.py rename to tests/integration/test_keeper_feature_flags_config/__init__.py diff --git a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml b/tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml similarity index 97% rename from tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml rename to tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml index c153d025598..53a169c4c3c 100644 --- a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml @@ -18,7 +18,7 @@ 0 - + diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py new file mode 100644 index 00000000000..432c413cbe8 --- /dev/null +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -0,0 +1,92 @@ +#!/usr/bin/env python3 + +import pytest +import os +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient, KazooState + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(feature_flags=[], expect_fail=True): + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + + if len(feature_flags) > 0: + feature_flags_config = "" + + for feature, is_enabled in feature_flags: + feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" + + feature_flags_config += "<\\/feature_flags>" + + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + feature_flags_config + ) + + node.start_clickhouse(retry_start=not expect_fail) + keeper_utils.wait_until_connected(cluster, node) + + +def test_keeper_feature_flags(started_cluster): + restart_clickhouse() + + def assert_feature_flags(feature_flags): + res = keeper_utils.send_4lw_cmd(started_cluster, node, "ftfl") + + for feature, is_enabled in feature_flags: + node.wait_for_log_line( + f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + ) + + node.wait_for_log_line( + f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + ) + + assert f"{feature}\t{1 if is_enabled else 0}" in res + + assert_feature_flags([("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)]) + + feature_flags = [("multi_read", 0), ("check_not_exists", 1)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags + [("filtered_list", 1)]) + + feature_flags = [("multi_read", 0), ("check_not_exists", 0), ("filtered_list", 0)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags) + + with pytest.raises(Exception): + restart_clickhouse([("invalid_feature", 1)], expect_fail=True) From 708a99a6ea63409ce33f83d450592eaa42411ebb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Jun 2023 11:19:38 +0000 Subject: [PATCH 04/79] Automatic style fix --- .../test_keeper_feature_flags_config/test.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py index 432c413cbe8..bb7252e9ec8 100644 --- a/tests/integration/test_keeper_feature_flags_config/test.py +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -45,7 +45,7 @@ def restart_clickhouse(feature_flags=[], expect_fail=True): if len(feature_flags) > 0: feature_flags_config = "" - + for feature, is_enabled in feature_flags: feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" @@ -54,7 +54,7 @@ def restart_clickhouse(feature_flags=[], expect_fail=True): node.replace_in_config( "/etc/clickhouse-server/config.d/enable_keeper.xml", "", - feature_flags_config + feature_flags_config, ) node.start_clickhouse(retry_start=not expect_fail) @@ -69,16 +69,20 @@ def test_keeper_feature_flags(started_cluster): for feature, is_enabled in feature_flags: node.wait_for_log_line( - f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, ) node.wait_for_log_line( - f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, ) assert f"{feature}\t{1 if is_enabled else 0}" in res - assert_feature_flags([("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)]) + assert_feature_flags( + [("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)] + ) feature_flags = [("multi_read", 0), ("check_not_exists", 1)] restart_clickhouse(feature_flags) From a7e6264d567ffa7456d00df017675f0acf4ca90f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 13:05:44 +0000 Subject: [PATCH 05/79] Add backward compatibility --- programs/keeper/CMakeLists.txt | 1 + src/Common/ZooKeeper/CMakeLists.txt | 2 ++ src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperConstants.h | 1 - src/Coordination/KeeperContext.cpp | 4 ++++ src/Storages/System/StorageSystemZooKeeperConnection.cpp | 4 ++-- 6 files changed, 10 insertions(+), 4 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 4f74cc06801..555c6431865 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -44,6 +44,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperLogStore.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperFeatureFlags.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index a9a335d1461..3f7e87ff4a7 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -2,6 +2,8 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_common_zookeeper .) +list(APPEND clickhouse_common_zookeeper_sources ${CMAKE_CURRENT_SOURCE_DIR}/../../../src/Coordination/KeeperFeatureFlags.cpp) + # for clickhouse server add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 55120e70d99..10d13657fb0 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -490,7 +490,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(0); + return toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } String CreateSnapshotCommand::run() diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 6d9c03ca8a9..84cbb0ab7c5 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -14,7 +14,6 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; - const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index dc408afc19a..e6f30c81310 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -11,6 +11,10 @@ KeeperContext::KeeperContext() feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); feature_flags.enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); + + + /// for older clients, the default is equivalent to WITH_MULTI_READ version + system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 9a6a592f2c4..99872be6ba0 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(0); + res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(0); + res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); res_columns[7]->insert(elem.second->getClientID()); } From 4ac090b12212567f7b9d30cb27132f566ae438c4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Jun 2023 11:01:44 +0000 Subject: [PATCH 06/79] Fix tests --- src/Coordination/tests/gtest_coordination.cpp | 22 +++++++++---------- .../StorageSystemZooKeeperConnection.cpp | 4 ++-- tests/config/config.d/keeper_port.xml | 4 ++++ .../test_keeper_four_word_command/test.py | 8 +++---- ...2735_system_zookeeper_connection.reference | 2 +- 5 files changed, 22 insertions(+), 18 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 2793b23c572..b29d5bca43d 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1193,7 +1193,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); EXPECT_EQ(snapshot.session_id, 7); - EXPECT_EQ(snapshot.snapshot_container_size, 5); + EXPECT_EQ(snapshot.snapshot_container_size, 6); EXPECT_EQ(snapshot.session_and_timeout.size(), 2); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -1205,7 +1205,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 5); + EXPECT_EQ(restored_storage->container.size(), 6); EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); @@ -1237,14 +1237,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) DB::KeeperStorageSnapshot snapshot(&storage, 50); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); - EXPECT_EQ(snapshot.snapshot_container_size, 53); + EXPECT_EQ(snapshot.snapshot_container_size, 54); for (size_t i = 50; i < 100; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - EXPECT_EQ(storage.container.size(), 103); + EXPECT_EQ(storage.container.size(), 104); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); @@ -1254,7 +1254,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) auto debuf = manager.deserializeSnapshotBufferFromDisk(50); auto [restored_storage, meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 53); + EXPECT_EQ(restored_storage->container.size(), 54); for (size_t i = 0; i < 50; ++i) { EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); @@ -1293,7 +1293,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); - EXPECT_EQ(restored_storage->container.size(), 253); + EXPECT_EQ(restored_storage->container.size(), 254); for (size_t i = 0; i < 250; ++i) { @@ -1327,16 +1327,16 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) if (i % 2 == 0) storage.container.erase("/hello_" + std::to_string(i)); } - EXPECT_EQ(storage.container.size(), 28); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 104); + EXPECT_EQ(storage.container.size(), 29); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 105); EXPECT_EQ(storage.container.snapshotSizeWithVersion().second, 1); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); - EXPECT_EQ(storage.container.size(), 28); + EXPECT_EQ(storage.container.size(), 29); storage.clearGarbageAfterSnapshot(); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 28); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 29); for (size_t i = 0; i < 50; ++i) { if (i % 2 != 0) @@ -1865,7 +1865,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 5); + EXPECT_EQ(restored_storage->container.size(), 6); EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 99872be6ba0..9a6a592f2c4 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); + res_columns[6]->insert(0); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); + res_columns[6]->insert(0); res_columns[7]->insert(elem.second->getClientID()); } diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index cffd325e968..7db174c5419 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -28,5 +28,9 @@ 9234 + + + 1 + diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 2098daea5fe..adc29abb584 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -183,8 +183,8 @@ def test_cmd_mntr(started_cluster): # contains: # 10 nodes created by test # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" - # 1 root node, 2 keeper system nodes - assert int(result["zk_znode_count"]) == 13 + # 1 root node, 3 keeper system nodes + assert int(result["zk_znode_count"]) == 14 assert int(result["zk_watch_count"]) == 2 assert int(result["zk_ephemerals_count"]) == 2 assert int(result["zk_approximate_data_size"]) > 0 @@ -329,7 +329,7 @@ def test_cmd_srvr(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"]) > 10 assert result["Mode"] == "leader" - assert result["Node count"] == "13" + assert result["Node count"] == "14" finally: destroy_zk_client(zk) @@ -369,7 +369,7 @@ def test_cmd_stat(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"]) >= 10 assert result["Mode"] == "leader" - assert result["Node count"] == "13" + assert result["Node count"] == "14" # filter connection statistics cons = [n for n in data.split("\n") if "=" in n] diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index 1deabd88b88..55b3579f0dd 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1,2 +1,2 @@ -default ::1 9181 0 0 3 +default ::1 9181 0 0 0 zookeeper2 ::1 9181 0 0 0 From 9a5c5c7e931c41f406e316cb20cc7659c40fccc5 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Tue, 13 Jun 2023 19:29:41 +0300 Subject: [PATCH 07/79] Update contrib/re2 to 2023-06-02 --- contrib/re2 | 2 +- src/Common/OptimizedRegularExpression.cpp | 12 ++++++------ src/Common/parseGlobs.cpp | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/contrib/re2 b/contrib/re2 index 13ebb377c6a..03da4fc0857 160000 --- a/contrib/re2 +++ b/contrib/re2 @@ -1 +1 @@ -Subproject commit 13ebb377c6ad763ca61d12dd6f88b1126bd0b911 +Subproject commit 03da4fc0857c285e3a26782f6bc8931c4c950df4 diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index f2fe922ef19..5df9ce76098 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -669,16 +669,16 @@ unsigned OptimizedRegularExpressionImpl::match(const char * subject matches.resize(limit); for (size_t i = 0; i < limit; ++i) { - if (pieces[i] != nullptr) - { - matches[i].offset = pieces[i].data() - subject; - matches[i].length = pieces[i].length(); - } - else + if (pieces[i].empty()) { matches[i].offset = std::string::npos; matches[i].length = 0; } + else + { + matches[i].offset = pieces[i].data() - subject; + matches[i].length = pieces[i].length(); + } } return limit; } diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index f8d331c2b76..07cce38afff 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -40,7 +40,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob size_t current_index = 0; while (RE2::FindAndConsume(&input, enum_or_range, &matched)) { - std::string buffer = matched.ToString(); + std::string buffer{matched}; oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; if (buffer.find(',') == std::string::npos) From 4096c082b9f7bd49664e6341e4862c1aed3d7a73 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Wed, 14 Jun 2023 15:09:17 +0300 Subject: [PATCH 08/79] Cleanup RE2_SOURCES --- contrib/re2-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/re2-cmake/CMakeLists.txt b/contrib/re2-cmake/CMakeLists.txt index 19939c11ebf..a081f92bc94 100644 --- a/contrib/re2-cmake/CMakeLists.txt +++ b/contrib/re2-cmake/CMakeLists.txt @@ -28,7 +28,6 @@ set(RE2_SOURCES ${SRC_DIR}/re2/regexp.cc ${SRC_DIR}/re2/set.cc ${SRC_DIR}/re2/simplify.cc - ${SRC_DIR}/re2/stringpiece.cc ${SRC_DIR}/re2/tostring.cc ${SRC_DIR}/re2/unicode_casefold.cc ${SRC_DIR}/re2/unicode_groups.cc From 003acb4fbe1f4a1704688cf4edf44183c09c65db Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 22 Jun 2023 18:31:01 +0200 Subject: [PATCH 09/79] add absl as include dir --- contrib/abseil-cpp-cmake/CMakeLists.txt | 14 ++++++++++++++ contrib/re2-cmake/CMakeLists.txt | 15 ++++----------- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index 4c31ecfc599..d64b6036611 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -17,3 +17,17 @@ get_target_property(FLAT_HASH_SET_INCLUDE_DIR absl::flat_hash_set INTERFACE_INCL target_include_directories (_abseil_swiss_tables SYSTEM BEFORE INTERFACE ${FLAT_HASH_SET_INCLUDE_DIR}) add_library(ch_contrib::abseil_swiss_tables ALIAS _abseil_swiss_tables) + +set(ABSL_FORMAT_SRC + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/arg.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/bind.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/extension.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/float_conversion.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/output.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/parser.cc +) + +add_library(_abseil_str_format ${ABSL_FORMAT_SRC}) +target_include_directories(_abseil_str_format PUBLIC ${ABSL_ROOT_DIR}) + +add_library(ch_contrib::abseil_str_format ALIAS _abseil_str_format) diff --git a/contrib/re2-cmake/CMakeLists.txt b/contrib/re2-cmake/CMakeLists.txt index a081f92bc94..305c2400c77 100644 --- a/contrib/re2-cmake/CMakeLists.txt +++ b/contrib/re2-cmake/CMakeLists.txt @@ -12,6 +12,7 @@ endif() set(SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/re2") set(RE2_SOURCES + ${SRC_DIR}/re2/bitmap256.cc ${SRC_DIR}/re2/bitstate.cc ${SRC_DIR}/re2/compile.cc ${SRC_DIR}/re2/dfa.cc @@ -31,11 +32,13 @@ set(RE2_SOURCES ${SRC_DIR}/re2/tostring.cc ${SRC_DIR}/re2/unicode_casefold.cc ${SRC_DIR}/re2/unicode_groups.cc + ${SRC_DIR}/util/pcre.cc ${SRC_DIR}/util/rune.cc ${SRC_DIR}/util/strutil.cc ) add_library(re2 ${RE2_SOURCES}) target_include_directories(re2 PUBLIC "${SRC_DIR}") +target_link_libraries(re2 ch_contrib::abseil_str_format) # Building re2 which is thread-safe and re2_st which is not. # re2 changes its state during matching of regular expression, e.g. creates temporary DFA. @@ -47,6 +50,7 @@ target_compile_definitions (re2_st PRIVATE NDEBUG NO_THREADS re2=re2_st) target_include_directories (re2_st PRIVATE .) target_include_directories (re2_st SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) target_include_directories (re2_st SYSTEM BEFORE PUBLIC ${SRC_DIR}) +target_link_libraries (re2_st ch_contrib::abseil_str_format) file (MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/re2_st) foreach (FILENAME filtered_re2.h re2.h set.h stringpiece.h) @@ -59,17 +63,6 @@ foreach (FILENAME filtered_re2.h re2.h set.h stringpiece.h) add_dependencies (re2_st transform_${FILENAME}) endforeach () -file (MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/util) -foreach (FILENAME mutex.h) - add_custom_command (OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/util/${FILENAME}" - COMMAND ${CMAKE_COMMAND} -DSOURCE_FILENAME="${SRC_DIR}/util/${FILENAME}" - -DTARGET_FILENAME="${CMAKE_CURRENT_BINARY_DIR}/util/${FILENAME}" - -P "${CMAKE_CURRENT_SOURCE_DIR}/re2_transform.cmake" - COMMENT "Creating ${FILENAME} for re2_st library.") - add_custom_target (transform_${FILENAME} DEPENDS "${CMAKE_CURRENT_BINARY_DIR}/util/${FILENAME}") - add_dependencies (re2_st transform_${FILENAME}) -endforeach () - # NOTE: you should not change name of library here, since it is used to generate required header (see above) add_library(ch_contrib::re2 ALIAS re2) add_library(ch_contrib::re2_st ALIAS re2_st) From fb7d23f245c3cd44f4f096f362fcb4ef948b1580 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 22 Jun 2023 18:31:12 +0200 Subject: [PATCH 10/79] fix build --- src/Dictionaries/RegExpTreeDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 3852cca6928..074b179c48e 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -479,7 +479,7 @@ std::pair processBackRefs(const String & data, const re2_st::RE2 & for (const auto & item : pieces) { if (item.ref_num >= 0 && item.ref_num < 10) - result += matches[item.ref_num].ToString(); + result += String{matches[item.ref_num]}; else result += item.literal; } From be17a817929ea7fef4a68ac108f89a65f0e21827 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 24 Jun 2023 00:12:31 +0000 Subject: [PATCH 11/79] recursively collect call stacks on job scheduling and add all stack traces to exceptions printout --- src/Common/Exception.cpp | 19 ++++++++++++++++++- src/Common/Exception.h | 12 +++++++++++- src/Common/ThreadPool.cpp | 9 ++++++++- src/Common/ThreadPool.h | 14 ++++++++++++-- 4 files changed, 49 insertions(+), 5 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 20206b76225..a43335ed8de 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -1,5 +1,6 @@ #include "Exception.h" +#include #include #include #include @@ -83,6 +84,7 @@ Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_) : Poco::Exception(msg_masked.msg, code) , remote(remote_) { + capture_thread_frame_pointers = thread_frame_pointers; handle_error_code(msg_masked.msg, code, remote, getStackFramePointers()); } @@ -90,12 +92,14 @@ Exception::Exception(MessageMasked && msg_masked, int code, bool remote_) : Poco::Exception(msg_masked.msg, code) , remote(remote_) { + capture_thread_frame_pointers = thread_frame_pointers; handle_error_code(message(), code, remote, getStackFramePointers()); } Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) { + capture_thread_frame_pointers = thread_frame_pointers; #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); auto stack_trace_size = exc.get_stack_trace_size(); @@ -107,6 +111,7 @@ Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) Exception::Exception(CreateFromSTDTag, const std::exception & exc) : Poco::Exception(demangle(typeid(exc).name()) + ": " + String(exc.what()), ErrorCodes::STD_EXCEPTION) { + capture_thread_frame_pointers = thread_frame_pointers; #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); auto stack_trace_size = exc.get_stack_trace_size(); @@ -153,7 +158,17 @@ std::string Exception::getStackTraceString() const auto * stack_trace_frames = get_stack_trace_frames(); auto stack_trace_size = get_stack_trace_size(); __msan_unpoison(stack_trace_frames, stack_trace_size * sizeof(stack_trace_frames[0])); - return StackTrace::toString(stack_trace_frames, 0, stack_trace_size); + String thread_stack_trace; + std::for_each(capture_thread_frame_pointers.rbegin(), capture_thread_frame_pointers.rend(), + [&thread_stack_trace](StackTrace::FramePointers & frame_pointers) + { + thread_stack_trace += + "\nJob's origin stack trace:\n" + + StackTrace::toString(&frame_pointers[0], 0, std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()); + } + ); + + return StackTrace::toString(stack_trace_frames, 0, stack_trace_size) + thread_stack_trace; #else return trace.toString(); #endif @@ -185,6 +200,8 @@ Exception::FramePointers Exception::getStackFramePointers() const return frame_pointers; } +thread_local std::vector Exception::thread_frame_pointers = {}; + void throwFromErrno(const std::string & s, int code, int the_errno) { diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 170e0d32b3c..c1bbeada506 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -25,18 +25,25 @@ class Exception : public Poco::Exception public: using FramePointers = std::vector; - Exception() = default; + Exception() + { + capture_thread_frame_pointers = thread_frame_pointers; + } Exception(const PreformattedMessage & msg, int code): Exception(msg.text, code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; } Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.text), code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; } + static thread_local std::vector thread_frame_pointers; + protected: // used to remove the sensitive information from exceptions if query_masking_rules is configured struct MessageMasked @@ -66,6 +73,7 @@ public: Exception(int code, T && message) : Exception(message, code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = tryGetStaticFormatString(message); } @@ -80,6 +88,7 @@ public: Exception(int code, FormatStringHelper fmt, Args &&... args) : Exception(fmt::format(fmt.fmt_str, std::forward(args)...), code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = fmt.message_format_string; } @@ -131,6 +140,7 @@ private: protected: std::string_view message_format_string; + mutable std::vector capture_thread_frame_pointers; }; diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index f91849ead66..0bd4fcb0455 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -189,7 +189,9 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: jobs.emplace(std::move(job), priority, /// Tracing context on this thread is used as parent context for the sub-thread that runs the job - propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread()); + propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread(), + /// capture_frame_pointers + true); ++scheduled_jobs; } @@ -348,6 +350,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// A copy of parent trace context DB::OpenTelemetry::TracingContextOnThread parent_thread_trace_context; + std::vector thread_frame_pointers; + /// Get a job from the queue. Job job; @@ -393,6 +397,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. job = std::move(const_cast(jobs.top().job)); parent_thread_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); + thread_frame_pointers = std::move(const_cast &>(jobs.top().frame_pointers)); jobs.pop(); /// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them. @@ -411,6 +416,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// Run the job. try { + DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); + CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); job(); diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 70053ff7558..17731de9540 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -19,6 +19,8 @@ #include #include #include +#include +#include #include /** Very simple thread pool similar to boost::threadpool. @@ -127,8 +129,16 @@ private: Priority priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_) - : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} + std::vector frame_pointers; + + JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, bool capture_frame_pointers = false) + : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) + { + if (!capture_frame_pointers) + return; + frame_pointers = DB::Exception::thread_frame_pointers; + frame_pointers.push_back(StackTrace().getFramePointers()); + } bool operator<(const JobWithPriority & rhs) const { From 7aa089902081922cb08e3e149f359a85add7785d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 07:55:38 +0200 Subject: [PATCH 12/79] Add a test for #42631 --- .../02803_remote_cannot_clone_block.reference | 1 + .../02803_remote_cannot_clone_block.sql | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/02803_remote_cannot_clone_block.reference create mode 100644 tests/queries/0_stateless/02803_remote_cannot_clone_block.sql diff --git a/tests/queries/0_stateless/02803_remote_cannot_clone_block.reference b/tests/queries/0_stateless/02803_remote_cannot_clone_block.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02803_remote_cannot_clone_block.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql new file mode 100644 index 00000000000..3e14aa6be96 --- /dev/null +++ b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS numbers_10_00223; + +CREATE TABLE numbers_10_00223 +ENGINE = Log AS +SELECT * +FROM system.numbers +LIMIT 10000; + +SELECT * +FROM +( + SELECT 1 + FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) + WITH TOTALS +) +WHERE 1 +GROUP BY 1; + +DROP TABLE numbers_10_00223; From f11e1c25ee2864bd06fbd1dd5bf6009ec0a391f9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 24 Jun 2023 17:24:54 +0000 Subject: [PATCH 13/79] added to fatal errors handler, added comments --- src/Common/Exception.h | 2 ++ src/Common/StackTrace.cpp | 15 +++++++++++++++ src/Common/StackTrace.h | 2 ++ src/Common/ThreadPool.h | 2 ++ src/Daemon/BaseDaemon.cpp | 32 +++++++++++++++++++++++++++++++- src/IO/ReadHelpers.h | 3 +++ src/IO/WriteHelpers.h | 3 +++ 7 files changed, 58 insertions(+), 1 deletion(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index c1bbeada506..4514df2159b 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -42,6 +42,7 @@ public: message_format_string = msg.format_string; } + /// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution static thread_local std::vector thread_frame_pointers; protected: @@ -140,6 +141,7 @@ private: protected: std::string_view message_format_string; + /// Local copy of static per-thread thread_frame_pointers, should be mutable to be unpoisoned on printout mutable std::vector capture_thread_frame_pointers; }; diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index e6b7839c4e2..aea0f854fe1 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -412,6 +412,21 @@ void StackTrace::toStringEveryLine(std::function callbac toStringEveryLineImpl(true, {frame_pointers, offset, size}, std::move(callback)); } +void StackTrace::toStringEveryLine(const FramePointers & frame_pointers, std::function callback) +{ + toStringEveryLineImpl(true, {frame_pointers, 0, static_cast(std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin())}, std::move(callback)); +} + +void StackTrace::toStringEveryLine(void ** frame_pointers_raw, size_t offset, size_t size, std::function callback) +{ + __msan_unpoison(frame_pointers_raw, size * sizeof(*frame_pointers_raw)); + + StackTrace::FramePointers frame_pointers{}; + std::copy_n(frame_pointers_raw, size, frame_pointers.begin()); + + toStringEveryLineImpl(true, {frame_pointers, offset, size}, std::move(callback)); +} + using StackTraceCache = std::map>; static StackTraceCache & cacheInstance() diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index 3940c880c5b..656f543d837 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -65,6 +65,8 @@ public: static void symbolize(const FramePointers & frame_pointers, size_t offset, size_t size, StackTrace::Frames & frames); void toStringEveryLine(std::function callback) const; + static void toStringEveryLine(const FramePointers & frame_pointers, std::function callback); + static void toStringEveryLine(void ** frame_pointers_raw, size_t offset, size_t size, std::function callback); /// Displaying the addresses can be disabled for security reasons. /// If you turn off addresses, it will be more secure, but we will be unable to help you with debugging. diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 17731de9540..57188572a9d 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -129,6 +129,7 @@ private: Priority priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; + /// Call stacks of all jobs' schedulings leading to this one std::vector frame_pointers; JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, bool capture_frame_pointers = false) @@ -136,6 +137,7 @@ private: { if (!capture_frame_pointers) return; + /// Save all previous jobs call stacks and append with current frame_pointers = DB::Exception::thread_frame_pointers; frame_pointers.push_back(StackTrace().getFramePointers()); } diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index bfd5568b71d..fb5b53201a2 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -153,6 +154,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) writePODBinary(*info, out); writePODBinary(signal_context, out); writePODBinary(stack_trace, out); + writeVectorBinary(Exception::thread_frame_pointers, out); writeBinary(static_cast(getThreadId()), out); writePODBinary(current_thread, out); @@ -250,6 +252,7 @@ public: siginfo_t info{}; ucontext_t * context{}; StackTrace stack_trace(NoCapture{}); + std::vector thread_frame_pointers; UInt32 thread_num{}; ThreadStatus * thread_ptr{}; @@ -260,12 +263,13 @@ public: } readPODBinary(stack_trace, in); + readVectorBinary(thread_frame_pointers, in); readBinary(thread_num, in); readPODBinary(thread_ptr, in); /// This allows to receive more signals if failure happens inside onFault function. /// Example: segfault while symbolizing stack trace. - std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, thread_ptr); }).detach(); + std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach(); } } } @@ -300,6 +304,7 @@ private: const siginfo_t & info, ucontext_t * context, const StackTrace & stack_trace, + const std::vector & thread_frame_pointers, UInt32 thread_num, ThreadStatus * thread_ptr) const { @@ -375,6 +380,31 @@ private: /// Write symbolized stack trace line by line for better grep-ability. stack_trace.toStringEveryLine([&](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + /// In case it's a scheduled job write all previous jobs origins call stacks + std::for_each(thread_frame_pointers.rbegin(), thread_frame_pointers.rend(), + [this](const StackTrace::FramePointers & frame_pointers) + { + if (size_t size = std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()) + { + LOG_FATAL(log, "========================================"); + WriteBufferFromOwnString bare_stacktrace; + writeString("Job's origin stack trace:", bare_stacktrace); + std::for_each_n(frame_pointers.begin(), size, + [&bare_stacktrace](const void * ptr) + { + writeChar(' ', bare_stacktrace); + writePointerHex(ptr, bare_stacktrace); + } + ); + + LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); + + StackTrace::toStringEveryLine(const_cast(&frame_pointers[0]), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + } + } + ); + + #if defined(OS_LINUX) /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. /// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index dabb21c046c..4a05a106de0 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -12,6 +12,7 @@ #include +#include #include #include #include @@ -1093,6 +1094,8 @@ inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf) inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); } + template inline void readBinaryEndian(T & x, ReadBuffer & buf) { diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 3d1a5aa49ef..ade7956a67d 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -10,6 +10,7 @@ #include +#include #include #include #include @@ -868,6 +869,8 @@ inline void writeBinary(const UUID & x, WriteBuffer & buf) { writePODBinary(x, b inline void writeBinary(const IPv4 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const IPv6 & x, WriteBuffer & buf) { writePODBinary(x, buf); } +inline void writeBinary(const StackTrace::FramePointers & x, WriteBuffer & buf) { writePODBinary(x, buf); } + /// Methods for outputting the value in text form for a tab-separated format. inline void writeText(is_integer auto x, WriteBuffer & buf) From e0485778f54e02d2f4b6ead1b1b3142023b569b1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 24 Jun 2023 19:37:00 +0000 Subject: [PATCH 14/79] style fix --- src/Daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index fb5b53201a2..f92dc82476d 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -403,7 +403,7 @@ private: } } ); - + #if defined(OS_LINUX) /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. From 8763c601d227a527566fe7f08647e86be205bde5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 25 Jun 2023 04:22:06 +0000 Subject: [PATCH 15/79] Automatic style fix --- tests/ci/fast_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index d377826513b..b316705245a 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -150,7 +150,7 @@ def main(): os.makedirs(logs_path) run_log_path = os.path.join(logs_path, "run.log") - with TeePopen(run_cmd, run_log_path, timeout = 90 * 60) as process: + with TeePopen(run_cmd, run_log_path, timeout=90 * 60) as process: retcode = process.wait() if retcode == 0: logging.info("Run successfully") From 535fef97d5d36180ac55ef76af3c480daa46c90c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 25 Jun 2023 04:44:21 +0000 Subject: [PATCH 16/79] clang-tidy fix --- src/Daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index f92dc82476d..634ef5e7f0a 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -399,7 +399,7 @@ private: LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); - StackTrace::toStringEveryLine(const_cast(&frame_pointers[0]), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + StackTrace::toStringEveryLine(const_cast(frame_pointers.data()), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); } } ); From cc3d27c0a30e40532541324b100f2584327b5ba3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 25 Jun 2023 15:14:29 +0000 Subject: [PATCH 17/79] clang-tidy fix --- src/Common/Exception.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index a43335ed8de..af48ce8fd99 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -164,7 +164,7 @@ std::string Exception::getStackTraceString() const { thread_stack_trace += "\nJob's origin stack trace:\n" + - StackTrace::toString(&frame_pointers[0], 0, std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()); + StackTrace::toString(frame_pointers.data(), 0, std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()); } ); From cf2e110c134c6934fd22e0c9ec5a598ac5b107cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:20:35 +0200 Subject: [PATCH 18/79] Fix test 01293_optimize_final_force --- tests/queries/0_stateless/01293_optimize_final_force.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01293_optimize_final_force.sh b/tests/queries/0_stateless/01293_optimize_final_force.sh index 994d5952dbc..eb3a2756899 100755 --- a/tests/queries/0_stateless/01293_optimize_final_force.sh +++ b/tests/queries/0_stateless/01293_optimize_final_force.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-debug, no-s3-storage +# This test is too slow with S3 storage and debug modes. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 179a7ce20263fd3bc50431ee234a99520595aebe Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sun, 25 Jun 2023 11:07:15 +0800 Subject: [PATCH 19/79] debug --- src/Interpreters/GraceHashJoin.cpp | 30 +++++++++++++++++++++++------- tests/ci/stress.py | 3 +++ 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 197b64865e1..4218a8ea4e1 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -385,11 +385,23 @@ GraceHashJoin::Buckets GraceHashJoin::rehashBuckets(size_t to_size) void GraceHashJoin::addBucket(Buckets & destination) { - auto & left_file = tmp_data->createStream(left_sample_block); - auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block)); + // There could be exceptions from createStream, In ci tests + // there is a certain probability of failure in allocating memory, see memory_tracker_fault_probability. + // It may terminate this thread and leave a broken hash_join, and another thread cores when it tries to + // use the broken hash_join. So we print an exception message here to help debug. + try + { + auto & left_file = tmp_data->createStream(left_sample_block); + auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block)); - BucketPtr new_bucket = std::make_shared(destination.size(), left_file, right_file, log); - destination.emplace_back(std::move(new_bucket)); + BucketPtr new_bucket = std::make_shared(destination.size(), left_file, right_file, log); + destination.emplace_back(std::move(new_bucket)); + } + catch (...) + { + LOG_ERROR(&Poco::Logger::get("GraceHashJoin"), "Can't create bucket. current buckets size: {}", destination.size()); + throw; + } } void GraceHashJoin::checkTypesOfKeys(const Block & block) const @@ -626,7 +638,11 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) if (current_block.rows() > 0) { std::lock_guard lock(hash_join_mutex); - + auto current_buckets = getCurrentBuckets(); + if (!isPowerOf2(current_buckets.size())) [[unlikely]] + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Broken buckets. its size({}) is not power of 2", current_buckets.size()); + } if (!hash_join) hash_join = makeInMemoryJoin(); @@ -637,11 +653,11 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = {}; + // Must use the latest buckets snapshot in case that it has been rehashed by other threads. + buckets_snapshot = rehashBuckets(current_buckets.size() * 2); auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); hash_join = nullptr; - buckets_snapshot = rehashBuckets(buckets_snapshot.size() * 2); - { Blocks current_blocks; current_blocks.reserve(right_blocks.size()); diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 1aa483854fc..6d17384c63f 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -38,6 +38,9 @@ def get_options(i, upgrade_check): client_options.append("join_algorithm='partial_merge'") if join_alg_num % 5 == 2: client_options.append("join_algorithm='full_sorting_merge'") + if join_alg_num % 5 == 3 and not upgrade_check: + # Some crashes are not fixed in 23.2 yet, so ignore the setting in Upgrade check + client_options.append("join_algorithm='grace_hash'") if join_alg_num % 5 == 4: client_options.append("join_algorithm='auto'") client_options.append("max_rows_in_join=1000") From aab86da4e9bc1840d740c00fa02c599a36c3d04c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 26 Jun 2023 20:42:21 +0800 Subject: [PATCH 20/79] increase max_bytes_in_join --- tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 98cc46c9cb4..7276e77dc16 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -30,7 +30,7 @@ SELECT 'skipped'; {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} -SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}1M{% else %}0{% endif %}'; +SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}16M{% else %}0{% endif %}'; SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; From b4b21c0cf2e0c9c128a1558b543787297059748a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 27 Jun 2023 10:05:14 +0200 Subject: [PATCH 21/79] Try fix flaky 02497_storage_file_reader_selection --- .../queries/0_stateless/02497_storage_file_reader_selection.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh index 20bde68718d..66b894404cf 100755 --- a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh +++ b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=test_02497_$CLICKHOUSE_TEST_UNIQUE_NAME.tsv echo -e 'key\nfoo\nbar' > $DATA_FILE -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap" +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap:" $CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" $CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap" From dc2c1a42267ec8c0edf86b3ebbad00d7bc4fa42b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 13:21:34 +0200 Subject: [PATCH 22/79] Convert 02003_memory_limit_in_client from expect to sh test (to fix flakiness) I don't see the reason for using expect here, and plus now the test is only 2 lines instead. Also this conversion should fix the test flakiness, since sometimes 60 seconds is not enough [1]: 2023.06.26 17:42:29.582168 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} executeQuery: Query span trace_id for opentelemetry log: 00000000-0000-0000-0000-000000000000 2023.06.26 17:42:29.584019 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} executeQuery: (from [::1]:38130) (comment: 02003_memory_limit_in_client.expect) SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) (stage: Complete) 2023.06.26 17:42:29.588131 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} InterpreterSelectQuery: FetchColumns -> Complete 2023.06.26 17:42:29.780980 [ 2566 ] {8a324061-a504-4843-be65-6abb5db4bc5c} MemoryTracker: Current memory usage (for query): 1.24 GiB. 2023.06.26 17:43:29.179098 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} TCPHandler: Client has dropped the connection, cancel the query. 2023.06.26 17:44:05.523345 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} executeQuery: Code: 394. DB::Exception: Query was cancelled or a client has unexpectedly dropped the connection. (QUERY_WAS_CANCELLED) (version 23.6.1.1) (from [::1]:38130) (comment: 02003_memory_limit_in_client.expect) (in query: SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)) 2023.06.26 17:44:05.680631 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} MemoryTracker: Peak memory usage (for query): 1.25 GiB. 2023.06.26 17:44:05.680747 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} TCPHandler: Processed in 96.108615451 sec. [1]: https://s3.amazonaws.com/clickhouse-test-reports/51407/e92035072b7b3367da12089b28041893eb90e636/stateless_tests__tsan__[2_5].html And it is either expect is too slow, or server did not respond for awhile, but this was a TSan build and trace_log is empty, so it is not possible to check which one was the problem. Signed-off-by: Azat Khuzhin --- .../02003_memory_limit_in_client.expect | 64 ------------------- .../02003_memory_limit_in_client.reference | 1 + .../02003_memory_limit_in_client.sh | 8 +++ 3 files changed, 9 insertions(+), 64 deletions(-) delete mode 100755 tests/queries/0_stateless/02003_memory_limit_in_client.expect create mode 100755 tests/queries/0_stateless/02003_memory_limit_in_client.sh diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect deleted file mode 100755 index 377656fa641..00000000000 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ /dev/null @@ -1,64 +0,0 @@ -#!/usr/bin/expect -f - -# This is a test for system.warnings. Testing in interactive mode is necessary, -# as we want to see certain warnings from client - -set basedir [file dirname $argv0] -set basename [file tail $argv0] -exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 -set history_file $env(CLICKHOUSE_TMP)/$basename.history - -log_user 0 -set timeout 60 -match_max 100000 - -expect_after { - # Do not ignore eof from expect - -i $any_spawn_id eof { exp_continue } - # A default timeout action is to do nothing, change it to fail - -i $any_spawn_id timeout { exit 1 } -} - -# -# Check that the query will fail in clickhouse-client -# -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" -expect ":) " - -send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" -expect "Code: 241" - -expect ":) " - -# Exit. -send -- "\4" -expect eof - -# -# Check that the query will fail in clickhouse-client -# -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" -expect ":) " - -send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" -expect "Code: 241" - -expect ":) " - -# Exit. -send -- "\4" -expect eof - -# -# Check that the query will not fail (due to max_untracked_memory) -# -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" -expect ":) " - -send -- "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000\r" -expect "60000" -expect ":) " - -# Exit. -send -- "\4" -expect eof diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.reference b/tests/queries/0_stateless/02003_memory_limit_in_client.reference index e69de29bb2d..541b3a18e90 100644 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.reference +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.reference @@ -0,0 +1 @@ +60000 diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh new file mode 100755 index 00000000000..2d2493828c8 --- /dev/null +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -0,0 +1,8 @@ +#!/usr/bin/bash -f + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" From acea2d66f0c4abad3c700ef2153977a5bcfba819 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 13:40:07 +0200 Subject: [PATCH 23/79] Fix --- tests/integration/test_disk_over_web_server/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 719de5e8bef..d62adfb3343 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -45,6 +45,8 @@ def cluster(): f"CREATE TABLE data{i} (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'def', min_bytes_for_wide_part=1;" ) + node.query("SYSTEM STOP MERGES") + for _ in range(10): node.query( f"INSERT INTO data{i} SELECT number FROM numbers(500000 * {i+1})" From 648b647f5f878c4da0265cfae3dc39e50021170b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 15:49:26 +0200 Subject: [PATCH 24/79] Fix race --- src/IO/BoundedReadBuffer.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/BoundedReadBuffer.h b/src/IO/BoundedReadBuffer.h index 183dbedb78b..eb65857e83a 100644 --- a/src/IO/BoundedReadBuffer.h +++ b/src/IO/BoundedReadBuffer.h @@ -31,7 +31,8 @@ public: private: std::optional read_until_position; - size_t file_offset_of_buffer_end = 0; + /// atomic because can be used in log or exception messages while being updated. + std::atomic file_offset_of_buffer_end = 0; }; } From b8866e01286c4e1adeb10554b823eb1a79b9119c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 16:11:27 +0200 Subject: [PATCH 25/79] Fix flaky unit test --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 3cba1e48e1e..429f940a7c7 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -475,7 +475,7 @@ TEST_F(FileCacheTest, get) } cv.notify_one(); - file_segment2.wait(file_segment2.range().left); + file_segment2.wait(file_segment2.range().right); file_segment2.complete(); ASSERT_TRUE(file_segment2.state() == State::DOWNLOADED); }); From 98721ca36c87379e6899837b46cf4b9b5810a969 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 27 Jun 2023 16:44:50 +0000 Subject: [PATCH 26/79] New epoch From 1ed104417a2cb49fc3c677d9193e2b676de95d77 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 27 Jun 2023 19:41:24 +0200 Subject: [PATCH 27/79] fix race between executeMetadataAlter and initializeReplication (cherry picked from commit 1d47783e857ed4dd7550de0728913e0144657a52) --- src/Databases/DatabaseReplicated.cpp | 10 ++++++++++ src/Databases/DatabaseReplicated.h | 2 ++ src/Databases/DatabaseReplicatedWorker.cpp | 5 ++++- src/Databases/IDatabase.h | 3 +++ src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 9 +++++++++ 5 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index de40ee4d82d..661afc6bf1f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1293,6 +1293,16 @@ void DatabaseReplicated::commitAlterTable(const StorageID & table_id, assert(checkDigestValid(query_context)); } + +bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const +{ + /// ReplicatedMergeTree may call commitAlterTable from its background threads when executing ALTER_METADATA entries. + /// It may update the metadata digest (both locally and in ZooKeeper) + /// before DatabaseReplicatedDDLWorker::initializeReplication() has finished. + /// We should not update metadata until the database is initialized. + return ddl_worker && ddl_worker->isCurrentlyActive(); +} + void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name) { auto txn = local_context->getZooKeeperMetadataTransaction(); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 1da181de030..ff1a4aba41c 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -48,6 +48,8 @@ public: /// then it will be executed on all replicas. BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context, bool internal) override; + bool canExecuteReplicatedMetadataAlter() const override; + bool hasReplicationThread() const override { return true; } void stopReplication() override; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index ff2675dfd6b..593d0655777 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -91,12 +91,12 @@ void DatabaseReplicatedDDLWorker::initializeReplication() if (zookeeper->tryGet(database->replica_path + "/digest", digest_str)) { digest = parse(digest_str); - LOG_TRACE(log, "Metadata digest in ZooKeeper: {}", digest); std::lock_guard lock{database->metadata_mutex}; local_digest = database->tables_metadata_digest; } else { + LOG_WARNING(log, "Did not find digest in ZooKeeper, creating it"); /// Database was created by old ClickHouse versions, let's create the node std::lock_guard lock{database->metadata_mutex}; digest = local_digest = database->tables_metadata_digest; @@ -104,6 +104,9 @@ void DatabaseReplicatedDDLWorker::initializeReplication() zookeeper->create(database->replica_path + "/digest", digest_str, zkutil::CreateMode::Persistent); } + LOG_TRACE(log, "Trying to initialize replication: our_log_ptr={}, max_log_ptr={}, local_digest={}, zk_digest={}", + our_log_ptr, max_log_ptr, local_digest, digest); + bool is_new_replica = our_log_ptr == 0; bool lost_according_to_log_ptr = our_log_ptr + logs_to_keep < max_log_ptr; bool lost_according_to_digest = database->db_settings.check_consistency && local_digest != digest; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 53a2f372814..aadae3e2491 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -254,6 +254,9 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: alterTable() is not supported", getEngineName()); } + /// Special method for ReplicatedMergeTree and DatabaseReplicated + virtual bool canExecuteReplicatedMetadataAlter() const { return true; } + /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. virtual time_t getObjectMetadataModificationTime(const String & /*name*/) const { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 3ba3048b812..792843cbe18 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1448,6 +1448,15 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( LOG_TRACE(LogToStr(out_postpone_reason, log), fmt_string, entry.znode_name, entry.alter_version, head_alter); return false; } + + auto database_name = storage.getStorageID().database_name; + auto database = DatabaseCatalog::instance().getDatabase(database_name); + if (!database->canExecuteReplicatedMetadataAlter()) + { + LOG_TRACE(LogToStr(out_postpone_reason, log), "Cannot execute alter metadata {} with version {} " + "because database {} cannot process metadata alters now", entry.znode_name, entry.alter_version, database_name); + return false; + } } /// If this MUTATE_PART is part of alter modify/drop query, than we have to execute them one by one From 6e769237a736d5f7c9e795451a4dddf675fdf85c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 28 Jun 2023 01:39:06 +0000 Subject: [PATCH 28/79] added setting 'enable_job_stack_trace', default is off --- src/Common/Exception.cpp | 1 + src/Common/Exception.h | 1 + src/Common/ThreadPool.cpp | 10 +++++++--- src/Common/ThreadPool.h | 3 ++- src/Common/ThreadStatus.h | 1 + src/Core/Settings.h | 1 + src/Interpreters/ThreadStatusExt.cpp | 14 ++++++++++++++ 7 files changed, 27 insertions(+), 4 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index af48ce8fd99..ee268be45f6 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -200,6 +200,7 @@ Exception::FramePointers Exception::getStackFramePointers() const return frame_pointers; } +thread_local bool Exception::enable_job_stack_trace = false; thread_local std::vector Exception::thread_frame_pointers = {}; diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 4514df2159b..f80dfe7f0a2 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -43,6 +43,7 @@ public: } /// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution + static thread_local bool enable_job_stack_trace; static thread_local std::vector thread_frame_pointers; protected: diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 0bd4fcb0455..979e53a72c0 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -191,7 +191,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: /// Tracing context on this thread is used as parent context for the sub-thread that runs the job propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread(), /// capture_frame_pointers - true); + DB::Exception::enable_job_stack_trace); ++scheduled_jobs; } @@ -397,7 +397,9 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. job = std::move(const_cast(jobs.top().job)); parent_thread_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); - thread_frame_pointers = std::move(const_cast &>(jobs.top().frame_pointers)); + DB::Exception::enable_job_stack_trace = jobs.top().enable_job_stack_trace; + if (DB::Exception::enable_job_stack_trace) + thread_frame_pointers = std::move(const_cast &>(jobs.top().frame_pointers)); jobs.pop(); /// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them. @@ -416,7 +418,9 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// Run the job. try { - DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); + if (DB::Exception::enable_job_stack_trace) + DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); + CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 57188572a9d..f5721146e09 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -131,9 +131,10 @@ private: /// Call stacks of all jobs' schedulings leading to this one std::vector frame_pointers; + bool enable_job_stack_trace = false; JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, bool capture_frame_pointers = false) - : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) + : job(job_), priority(priority_), thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers) { if (!capture_frame_pointers) return; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 061959d9f1f..3b6b947471e 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -290,6 +290,7 @@ public: void flushUntrackedMemory(); private: + void applyGlobalSettings(); void applyQuerySettings(); void initPerformanceCounters(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c14bd420c5c..d4003124303 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -465,6 +465,7 @@ class IColumn; M(UInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \ M(UInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).", 0) \ M(Bool, calculate_text_stack_trace, true, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.", 0) \ + M(Bool, enable_job_stack_trace, false, "Output stack trace of a job creator when job results in exception", 0) \ M(Bool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.", 0) \ M(Bool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.", 0) \ M(Bool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \ diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 6a4f4576eca..5acfe500b1d 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -158,6 +158,17 @@ void CurrentThread::attachQueryForLog(const String & query_) current_thread->attachQueryForLog(query_); } +void ThreadStatus::applyGlobalSettings() +{ + auto global_context_ptr = global_context.lock(); + if (!global_context_ptr) + return; + + const Settings & settings = global_context_ptr->getSettingsRef(); + + DB::Exception::enable_job_stack_trace = settings.enable_job_stack_trace; +} + void ThreadStatus::applyQuerySettings() { auto query_context_ptr = query_context.lock(); @@ -166,6 +177,8 @@ void ThreadStatus::applyQuerySettings() const Settings & settings = query_context_ptr->getSettingsRef(); + DB::Exception::enable_job_stack_trace = settings.enable_job_stack_trace; + query_id_from_query_context = query_context_ptr->getCurrentQueryId(); initQueryProfiler(); @@ -204,6 +217,7 @@ void ThreadStatus::attachToGroupImpl(const ThreadGroupPtr & thread_group_) local_data = thread_group->getSharedData(); + applyGlobalSettings(); applyQuerySettings(); initPerformanceCounters(); } From be852d554bb592cb66cacd0feef75b496b2465f1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 27 Jun 2023 13:33:39 +0000 Subject: [PATCH 29/79] Use magic_enum for values --- src/Common/ZooKeeper/IKeeper.h | 2 + src/Common/ZooKeeper/ZooKeeper.h | 2 + src/Common/ZooKeeper/ZooKeeperImpl.h | 2 + src/Coordination/FourLetterCommand.cpp | 11 +++-- src/Coordination/KeeperContext.cpp | 20 +++++++-- src/Coordination/KeeperFeatureFlags.cpp | 17 +++----- src/Coordination/KeeperFeatureFlags.h | 14 +------ .../StorageSystemZooKeeperConnection.cpp | 41 ++++++++++++++++--- .../test_keeper_feature_flags_config/test.py | 4 +- ...2735_system_zookeeper_connection.reference | 4 +- .../02735_system_zookeeper_connection.sql | 2 +- 11 files changed, 80 insertions(+), 39 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 369aacf16c7..2703c1079c0 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -532,6 +532,8 @@ public: virtual bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const = 0; + virtual const DB::KeeperFeatureFlags * getKeeperFeatureFlags() const { return nullptr; } + /// Expire session and finish all pending requests virtual void finalize(const String & reason) = 0; }; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 03200771e4a..1fcb048add2 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -529,6 +529,8 @@ public: size_t getConnectedZooKeeperIndex() const { return connected_zk_index; } UInt64 getConnectedTime() const { return connected_time; } + const DB::KeeperFeatureFlags * getKeeperFeatureFlags() const { return impl->getKeeperFeatureFlags(); } + private: void init(ZooKeeperArgs args_); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index ae6bef067e3..44ea993947e 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -202,6 +202,8 @@ public: void setServerCompletelyStarted(); + const KeeperFeatureFlags * getKeeperFeatureFlags() const override { return &keeper_feature_flags; } + private: ACLs default_acls; Poco::Net::SocketAddress connected_zk_address; diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 10d13657fb0..34540902d47 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -545,7 +546,7 @@ String FeatureFlagsCommand::run() StringBuffer ret; - auto append = [&ret] (String key, uint8_t value) -> void + auto append = [&ret] (const String & key, uint8_t value) -> void { writeText(key, ret); writeText('\t', ret); @@ -553,8 +554,12 @@ String FeatureFlagsCommand::run() writeText('\n', ret); }; - for (const auto feature : all_keeper_feature_flags) - append(SettingFieldKeeperFeatureFlagTraits::toString(feature), feature_flags.isEnabled(feature)); + for (const auto & [feature_flag, name] : magic_enum::enum_entries()) + { + std::string feature_flag_string(name); + boost::to_lower(feature_flag_string); + append(feature_flag_string, feature_flags.isEnabled(feature_flag)); + } return ret.str(); diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index e6f30c81310..e1c3a138646 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -1,10 +1,19 @@ #include #include #include +#include +#include namespace DB { +namespace ErrorCodes +{ + +extern const int BAD_ARGUMENTS; + +} + KeeperContext::KeeperContext() { /// enable by default some feature flags @@ -29,12 +38,17 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) config.keys(feature_flags_key, keys); for (const auto & key : keys) { - auto feature_flag = SettingFieldKeeperFeatureFlagTraits::fromString(key); + auto feature_flag_string = boost::to_upper_copy(key); + auto feature_flag = magic_enum::enum_cast(feature_flag_string); + + if (!feature_flag.has_value()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid feature flag defined in config for Keeper: {}", key); + auto is_enabled = config.getBool(feature_flags_key + "." + key); if (is_enabled) - feature_flags.enableFeatureFlag(feature_flag); + feature_flags.enableFeatureFlag(feature_flag.value()); else - feature_flags.disableFeatureFlag(feature_flag); + feature_flags.disableFeatureFlag(feature_flag.value()); } system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp index 216dca014d4..3aff87bcea9 100644 --- a/src/Coordination/KeeperFeatureFlags.cpp +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -23,15 +23,10 @@ std::pair getByteAndBitIndex(size_t num) } -IMPLEMENT_SETTING_ENUM(KeeperFeatureFlag, ErrorCodes::BAD_ARGUMENTS, - {{"filtered_list", KeeperFeatureFlag::FILTERED_LIST}, - {"multi_read", KeeperFeatureFlag::MULTI_READ}, - {"check_not_exists", KeeperFeatureFlag::CHECK_NOT_EXISTS}}); - KeeperFeatureFlags::KeeperFeatureFlags() { /// get byte idx of largest value - auto [byte_idx, _] = getByteAndBitIndex(all_keeper_feature_flags.size() - 1); + auto [byte_idx, _] = getByteAndBitIndex(magic_enum::enum_count() - 1); feature_flags = std::string(byte_idx + 1, 0); } @@ -56,7 +51,7 @@ void KeeperFeatureFlags::fromApiVersion(KeeperApiVersion keeper_api_version) bool KeeperFeatureFlags::isEnabled(KeeperFeatureFlag feature_flag) const { - auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + auto [byte_idx, bit_idx] = getByteAndBitIndex(magic_enum::enum_integer(feature_flag)); if (byte_idx > feature_flags.size()) return false; @@ -71,7 +66,7 @@ void KeeperFeatureFlags::setFeatureFlags(std::string feature_flags_) void KeeperFeatureFlags::enableFeatureFlag(KeeperFeatureFlag feature_flag) { - auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + auto [byte_idx, bit_idx] = getByteAndBitIndex(magic_enum::enum_integer(feature_flag)); chassert(byte_idx < feature_flags.size()); feature_flags[byte_idx] |= (1 << bit_idx); @@ -79,7 +74,7 @@ void KeeperFeatureFlags::enableFeatureFlag(KeeperFeatureFlag feature_flag) void KeeperFeatureFlags::disableFeatureFlag(KeeperFeatureFlag feature_flag) { - auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + auto [byte_idx, bit_idx] = getByteAndBitIndex(magic_enum::enum_integer(feature_flag)); chassert(byte_idx < feature_flags.size()); feature_flags[byte_idx] &= ~(1 << bit_idx); @@ -92,10 +87,10 @@ const std::string & KeeperFeatureFlags::getFeatureFlags() const void KeeperFeatureFlags::logFlags(Poco::Logger * log) const { - for (const auto & feature_flag : all_keeper_feature_flags) + for (const auto & [feature_flag, feature_flag_name] : magic_enum::enum_entries()) { auto is_enabled = isEnabled(feature_flag); - LOG_INFO(log, "Keeper feature flag {}: {}", SettingFieldKeeperFeatureFlagTraits::toString(feature_flag), is_enabled ? "enabled" : "disabled"); + LOG_INFO(log, "Keeper feature flag {}: {}", feature_flag_name, is_enabled ? "enabled" : "disabled"); } } diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Coordination/KeeperFeatureFlags.h index cdd4704a7ca..6c48915f60c 100644 --- a/src/Coordination/KeeperFeatureFlags.h +++ b/src/Coordination/KeeperFeatureFlags.h @@ -1,28 +1,18 @@ #pragma once -#include -#include #include namespace DB { -enum KeeperFeatureFlag +/// these values cannot be reordered or removed, only new values can be added +enum class KeeperFeatureFlag : size_t { FILTERED_LIST = 0, MULTI_READ, CHECK_NOT_EXISTS, }; -static inline constexpr std::array all_keeper_feature_flags -{ - KeeperFeatureFlag::FILTERED_LIST, - KeeperFeatureFlag::MULTI_READ, - KeeperFeatureFlag::CHECK_NOT_EXISTS, -}; - -DECLARE_SETTING_ENUM(KeeperFeatureFlag); - class KeeperFeatureFlags { public: diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index cd78ae01457..33268d58358 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -1,8 +1,11 @@ #include +#include +#include #include #include #include #include +#include #include namespace DB @@ -10,6 +13,13 @@ namespace DB NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() { + DataTypeEnum16::Values feature_flags_enum_values; + feature_flags_enum_values.reserve(magic_enum::enum_count()); + for (const auto & [feature_flag, feature_flag_string] : magic_enum::enum_entries()) + feature_flags_enum_values.push_back(std::pair{std::string{feature_flag_string}, static_cast(feature_flag)}); + + auto feature_flags_enum = std::make_shared(std::move(feature_flags_enum_values)); + return { {"name", std::make_shared()}, {"host", std::make_shared()}, @@ -19,7 +29,8 @@ NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() {"session_uptime_elapsed_seconds", std::make_shared()}, {"is_expired", std::make_shared()}, {"keeper_api_version", std::make_shared()}, - {"client_id", std::make_shared()} + {"client_id", std::make_shared()}, + {"enabled_feature_flags", std::make_shared(std::move(feature_flags_enum))} }; } @@ -36,17 +47,37 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[7]->insert(0); res_columns[8]->insert(context->getZooKeeper()->getClientID()); + const auto add_enabled_feature_flags = [&](const auto & zookeeper) + { + Array enabled_feature_flags; + const auto * feature_flags = zookeeper->getKeeperFeatureFlags(); + if (feature_flags) + { + for (const auto & feature_flag : magic_enum::enum_values()) + { + if (feature_flags->isEnabled(feature_flag)) + { + enabled_feature_flags.push_back(feature_flag); + } + } + } + res_columns[9]->insert(std::move(enabled_feature_flags)); + }; + + add_enabled_feature_flags(context->getZooKeeper()); + for (const auto & elem : context->getAuxiliaryZooKeepers()) { res_columns[0]->insert(elem.first); res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); res_columns[2]->insert(elem.second->getConnectedZooKeeperPort()); res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); - res_columns[4]->insert(elem.second->getSessionUptime()); - res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(0); - res_columns[7]->insert(elem.second->getClientID()); + res_columns[4]->insert(elem.second->getConnectedTime()); + res_columns[5]->insert(elem.second->getSessionUptime()); + res_columns[6]->insert(elem.second->expired()); + res_columns[7]->insert(0); res_columns[8]->insert(elem.second->getClientID()); + add_enabled_feature_flags(elem.second); } } diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py index bb7252e9ec8..93ac6cbd3bd 100644 --- a/tests/integration/test_keeper_feature_flags_config/test.py +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -69,12 +69,12 @@ def test_keeper_feature_flags(started_cluster): for feature, is_enabled in feature_flags: node.wait_for_log_line( - f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + f"ZooKeeperClient: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000, ) node.wait_for_log_line( - f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + f"KeeperContext: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000, ) diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index eddd4829829..380da27cde6 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1,2 +1,2 @@ -default ::1 9181 0 0 0 1 1 -zookeeper2 ::1 9181 0 0 0 1 \ No newline at end of file +default ::1 9181 0 0 0 1 1 ['FILTERED_LIST','MULTI_READ','CHECK_NOT_EXISTS'] +zookeeper2 ::1 9181 0 0 0 1 diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql index 863d90e1654..f999da51225 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -9,7 +9,7 @@ ENGINE ReplicatedMergeTree('zookeeper2:/clickhouse/{database}/02731_zk_connectio ORDER BY tuple(); select name, host, port, index, is_expired, keeper_api_version, (connected_time between yesterday() and now()), - (abs(session_uptime_elapsed_seconds - zookeeperSessionUptime()) < 10) + (abs(session_uptime_elapsed_seconds - zookeeperSessionUptime()) < 10), enabled_feature_flags from system.zookeeper_connection where name='default'; -- keeper_api_version will by 0 for auxiliary_zookeeper2, because we fail to get /api_version due to chroot From eff1cc0e5df2af76b5687fce03901686e1a8360c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 28 Jun 2023 10:20:33 +0200 Subject: [PATCH 30/79] Remove unused errorcode --- src/Coordination/KeeperFeatureFlags.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp index 3aff87bcea9..d0cd1c86b55 100644 --- a/src/Coordination/KeeperFeatureFlags.cpp +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - namespace { From 0e72c6dcc934a52184909e3c6b32b084678c185a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 10:33:07 +0200 Subject: [PATCH 31/79] fix --- .../0_stateless/01111_create_drop_replicated_db_stress.sh | 2 +- tests/queries/0_stateless/01293_optimize_final_force.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 770a0780ca2..4d341e5b8a3 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -64,7 +64,7 @@ function alter_table() if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ "alter table $table update n = n + (select max(n) from merge(REGEXP('${CLICKHOUSE_DATABASE}.*'), '.*')) where 1 settings allow_nondeterministic_mutations=1" \ - 2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY | grep -Fv TABLE_IS_DROPPED + 2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY | grep -Fv TABLE_IS_DROPPED | grep -Fv "Error while executing table function merge" sleep 0.$RANDOM done } diff --git a/tests/queries/0_stateless/01293_optimize_final_force.sh b/tests/queries/0_stateless/01293_optimize_final_force.sh index eb3a2756899..9b9ed6272a1 100755 --- a/tests/queries/0_stateless/01293_optimize_final_force.sh +++ b/tests/queries/0_stateless/01293_optimize_final_force.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-debug, no-s3-storage +# Tags: no-fasttest, long, no-debug, no-s3-storage # This test is too slow with S3 storage and debug modes. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 451694d8b6ed7630842020ca132a9d7548352cad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Jun 2023 10:24:54 +0000 Subject: [PATCH 32/79] Make `test_ssl_cert_authentication` similar to `test_tlvs1_3` --- .../test_ssl_cert_authentication/test.py | 92 +++++++++++++------ 1 file changed, 62 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index b05a6acc16b..91faf8b0ce3 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -2,10 +2,10 @@ import pytest from helpers.client import Client from helpers.cluster import ClickHouseCluster from helpers.ssl_context import WrapSSLContextWithSNI +import urllib.request, urllib.parse import ssl import os.path from os import remove -import urllib3 # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. @@ -14,6 +14,7 @@ SSL_HOST = "integration-tests.clickhouse.com" HTTPS_PORT = 8443 # It's important for the node to work at this IP because 'server-cert.pem' requires that (see server-ext.cnf). SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +MAX_RETRY = 5 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( @@ -164,17 +165,19 @@ def get_ssl_context(cert_name): def execute_query_https( query, user, enable_ssl_auth=True, cert_name=None, password=None ): - url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}" - headers = {"X-ClickHouse-User": user} + url = ( + f"https://{instance.ip_address}:{HTTPS_PORT}/?query={urllib.parse.quote(query)}" + ) + request = urllib.request.Request(url) + request.add_header("X-ClickHouse-User", user) if enable_ssl_auth: - headers["X-ClickHouse-SSL-Certificate-Auth"] = "on" + request.add_header("X-ClickHouse-SSL-Certificate-Auth", "on") if password: - headers["X-ClickHouse-Key"] = password - http_client = urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) - response = http_client.request("GET", url, headers=headers) - if response.status != 200: - raise Exception(response.status) - return response.data.decode("utf-8") + request.add_header("X-ClickHouse-Key", password) + response = urllib.request.urlopen( + request, context=get_ssl_context(cert_name) + ).read() + return response.decode("utf-8") def test_https(): @@ -198,10 +201,18 @@ def test_https_wrong_cert(): execute_query_https("SELECT currentUser()", user="john", cert_name="client2") assert "403" in str(err.value) + count = 0 # Wrong certificate: self-signed certificate. - with pytest.raises(Exception) as err: - execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, err: {err_str}") + continue + assert "unknown ca" in err_str + break # No certificate. with pytest.raises(Exception) as err: @@ -291,24 +302,45 @@ def test_https_non_ssl_auth(): == "jane\n" ) + count = 0 # However if we send a certificate it must not be wrong. - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="peter", - enable_ssl_auth=False, - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="jane", - enable_ssl_auth=False, - password="qwe123", - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="peter", + enable_ssl_auth=False, + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning( + f"Failed attempt with wrong cert, user: peter, err: {err_str}" + ) + continue + assert "unknown ca" in err_str + break + + count = 0 + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="jane", + enable_ssl_auth=False, + password="qwe123", + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning( + f"Failed attempt with wrong cert, user: jane, err: {err_str}" + ) + continue + assert "unknown ca" in err_str + break def test_create_user(): From d3677adcdf367ec87aba49ec844c607a9d6c987d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Jun 2023 10:37:38 +0000 Subject: [PATCH 33/79] Import missing module --- tests/integration/test_ssl_cert_authentication/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 91faf8b0ce3..ff2de7491e1 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -6,6 +6,7 @@ import urllib.request, urllib.parse import ssl import os.path from os import remove +import logging # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. From 3c8c4ce449fb9bfa74de74a0756b85f9c149efbe Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:16:26 +0200 Subject: [PATCH 34/79] Update test.py --- tests/integration/test_storage_postgresql/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index d60a90ed7ce..05d4b1e12ef 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -335,7 +335,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 + assert count <= 19 busy_pool = Pool(30) p = busy_pool.map_async(node_insert_select, range(30)) @@ -347,7 +347,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 + assert count <= 20 node1.query("DROP TABLE test.test_table;") node1.query("DROP TABLE test.stat;") From 06206d092d843fd3e6c005609008cd85597f9124 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:18:53 +0200 Subject: [PATCH 35/79] Update test.py --- tests/integration/test_storage_postgresql/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 05d4b1e12ef..83eb4f6f02a 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -323,7 +323,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 + assert count <= 18 # 16 for test.test_table + 1 for conn + 1 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert, range(30)) @@ -335,7 +335,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 19 + assert count <= 19 # 16 for test.test_table + 1 for conn + at most 2 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert_select, range(30)) @@ -347,7 +347,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 20 + assert count <= 20 # 16 for test.test_table + 1 for conn + at most 3 for test.stat node1.query("DROP TABLE test.test_table;") node1.query("DROP TABLE test.stat;") From 5443987113e0417c310eaf7bbfdfdf26de64f142 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 28 Jun 2023 11:33:45 +0000 Subject: [PATCH 36/79] Automatic style fix --- tests/integration/test_storage_postgresql/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 83eb4f6f02a..9f7c012e66f 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -323,7 +323,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 # 16 for test.test_table + 1 for conn + 1 for test.stat + assert count <= 18 # 16 for test.test_table + 1 for conn + 1 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert, range(30)) @@ -335,7 +335,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 19 # 16 for test.test_table + 1 for conn + at most 2 for test.stat + assert count <= 19 # 16 for test.test_table + 1 for conn + at most 2 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert_select, range(30)) @@ -347,7 +347,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 20 # 16 for test.test_table + 1 for conn + at most 3 for test.stat + assert count <= 20 # 16 for test.test_table + 1 for conn + at most 3 for test.stat node1.query("DROP TABLE test.test_table;") node1.query("DROP TABLE test.stat;") From f57d91b678226094bc55074b3df1ce8ba630fb5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Jun 2023 17:14:58 +0300 Subject: [PATCH 37/79] Update 02803_remote_cannot_clone_block.sql --- tests/queries/0_stateless/02803_remote_cannot_clone_block.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql index 3e14aa6be96..6d79aa76d18 100644 --- a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql +++ b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql @@ -6,6 +6,8 @@ SELECT * FROM system.numbers LIMIT 10000; +SET allow_experimental_analyzer = 0; + SELECT * FROM ( From 0b19c1832a7afd9015e1872a932b2d28b326117c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 28 Jun 2023 14:15:03 +0000 Subject: [PATCH 38/79] Fix: detach from thread group --- src/Dictionaries/HashedDictionary.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index eb1d98a8f39..798f37cb516 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include @@ -69,6 +70,11 @@ public: shards_queues[shard].emplace(backlog); pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()] { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + /// Do not account memory that was occupied by the dictionaries for the query/user context. MemoryTrackerBlockerInThread memory_blocker; From 56354b72514e79dfcd9a322c4cbf2aa2e81e6892 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 28 Jun 2023 16:55:22 +0000 Subject: [PATCH 39/79] Fix yet another place --- src/Dictionaries/HashedDictionary.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 798f37cb516..5f25600db8f 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -236,6 +236,11 @@ HashedDictionary::~HashedDictionary() pool.trySchedule([&container, thread_group = CurrentThread::getGroup()] { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + /// Do not account memory that was occupied by the dictionaries for the query/user context. MemoryTrackerBlockerInThread memory_blocker; From eea3c39959876a78639d53d4a5f84354cc53135b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 10:49:46 +0000 Subject: [PATCH 40/79] Cosmetics --- .../functions/tuple-functions.md | 7 +- src/Common/assert_cast.h | 2 +- src/Functions/tupleElement.cpp | 140 +++++++----------- .../0_stateless/02116_tuple_element.sql | 22 +-- .../02354_tuple_element_with_default.sql | 10 +- 5 files changed, 73 insertions(+), 108 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 1739920c9f0..7ed2deaeda6 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -22,14 +22,15 @@ tuple(x, y, …) A function that allows getting a column from a tuple. -If the second argument is a number `n`, it is the column index, starting from 1. If the second argument is a string `s`, it represents the name of the element. Besides, we can provide the third optional argument, such that when index out of bounds or element for such name does not exist, the default value returned instead of throw exception. The second and third arguments if provided are always must be constant. There is no cost to execute the function. +If the second argument is a number `index`, it is the column index, starting from 1. If the second argument is a string `name`, it represents the name of the element. Besides, we can provide the third optional argument, such that when index out of bounds or no element exist for the name, the default value returned instead of throwing an exception. The second and third arguments, if provided, must be constants. There is no cost to execute the function. -The function implements the operator `x.n` and `x.s`. +The function implements operators `x.index` and `x.name`. **Syntax** ``` sql -tupleElement(tuple, n/s [, default_value]) +tupleElement(tuple, index, [, default_value]) +tupleElement(tuple, name, [, default_value]) ``` ## untuple diff --git a/src/Common/assert_cast.h b/src/Common/assert_cast.h index 604cfaed6e2..0b73ba1cc12 100644 --- a/src/Common/assert_cast.h +++ b/src/Common/assert_cast.h @@ -23,7 +23,7 @@ namespace DB * The exact match of the type is checked. That is, cast to the ancestor will be unsuccessful. */ template -To assert_cast(From && from) +inline To assert_cast(From && from) { #ifndef NDEBUG try diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index b1fd200f5cd..fb8f1d3b48d 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -34,32 +34,14 @@ class FunctionTupleElement : public IFunction { public: static constexpr auto name = "tupleElement"; - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } bool isVariadic() const override { return true; } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - bool useDefaultImplementationForNulls() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override @@ -72,107 +54,98 @@ public: getName(), number_of_arguments); size_t count_arrays = 0; - const IDataType * tuple_col = arguments[0].type.get(); - while (const DataTypeArray * array = checkAndGetDataType(tuple_col)) + const IDataType * input_type = arguments[0].type.get(); + while (const DataTypeArray * array = checkAndGetDataType(input_type)) { - tuple_col = array->getNestedType().get(); + input_type = array->getNestedType().get(); ++count_arrays; } - const DataTypeTuple * tuple = checkAndGetDataType(tuple_col); + const DataTypeTuple * tuple = checkAndGetDataType(input_type); if (!tuple) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be tuple or array of tuple. Actual {}", getName(), arguments[0].type->getName()); - auto index = getElementNum(arguments[1].column, *tuple, number_of_arguments); + std::optional index = getElementIndex(arguments[1].column, *tuple, number_of_arguments); if (index.has_value()) { - DataTypePtr out_return_type = tuple->getElements()[index.value()]; + DataTypePtr return_type = tuple->getElements()[index.value()]; for (; count_arrays; --count_arrays) - out_return_type = std::make_shared(out_return_type); + return_type = std::make_shared(return_type); - return out_return_type; + return return_type; } else { - const IDataType * default_col = arguments[2].type.get(); - size_t default_argument_count_arrays = 0; - if (const DataTypeArray * array = checkAndGetDataType(default_col)) - { - default_argument_count_arrays = array->getNumberOfDimensions(); - } + const IDataType * default_type = arguments[2].type.get(); + size_t default_count_arrays = 0; - if (count_arrays != default_argument_count_arrays) - { + if (const DataTypeArray * default_type_as_array = checkAndGetDataType(default_type)) + default_count_arrays = default_type_as_array->getNumberOfDimensions(); + + if (count_arrays != default_count_arrays) throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, "Dimension of types mismatched between first argument and third argument. " "Dimension of 1st argument: {}. " - "Dimension of 3rd argument: {}.",count_arrays, default_argument_count_arrays); - } + "Dimension of 3rd argument: {}", count_arrays, default_count_arrays); + return arguments[2].type; } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - Columns array_offsets; + const auto & input_arg = arguments[0]; + const IDataType * input_type = input_arg.type.get(); + const IColumn * input_col = input_arg.column.get(); - const auto & first_arg = arguments[0]; - - const IDataType * tuple_type = first_arg.type.get(); - const IColumn * tuple_col = first_arg.column.get(); - bool first_arg_is_const = false; - if (typeid_cast(tuple_col)) + bool input_arg_is_const = false; + if (typeid_cast(input_col)) { - tuple_col = assert_cast(tuple_col)->getDataColumnPtr().get(); - first_arg_is_const = true; + input_col = assert_cast(input_col)->getDataColumnPtr().get(); + input_arg_is_const = true; } - while (const DataTypeArray * array_type = checkAndGetDataType(tuple_type)) - { - const ColumnArray * array_col = assert_cast(tuple_col); - tuple_type = array_type->getNestedType().get(); - tuple_col = &array_col->getData(); + Columns array_offsets; + while (const DataTypeArray * array_type = checkAndGetDataType(input_type)) + { + const ColumnArray * array_col = assert_cast(input_col); + + input_type = array_type->getNestedType().get(); + input_col = &array_col->getData(); array_offsets.push_back(array_col->getOffsetsPtr()); } - const DataTypeTuple * tuple_type_concrete = checkAndGetDataType(tuple_type); - const ColumnTuple * tuple_col_concrete = checkAndGetColumn(tuple_col); - if (!tuple_type_concrete || !tuple_col_concrete) + const DataTypeTuple * input_type_as_tuple = checkAndGetDataType(input_type); + const ColumnTuple * input_col_as_tuple = checkAndGetColumn(input_col); + if (!input_type_as_tuple || !input_col_as_tuple) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument for function {} must be tuple or array of tuple. Actual {}", - getName(), - first_arg.type->getName()); + "First argument for function {} must be tuple or array of tuple. Actual {}", getName(), input_arg.type->getName()); - auto index = getElementNum(arguments[1].column, *tuple_type_concrete, arguments.size()); + std::optional index = getElementIndex(arguments[1].column, *input_type_as_tuple, arguments.size()); if (!index.has_value()) { if (!array_offsets.empty()) - { recursiveCheckArrayOffsets(arguments[0].column, arguments[2].column, array_offsets.size()); - } return arguments[2].column; } - ColumnPtr res = tuple_col_concrete->getColumns()[index.value()]; + ColumnPtr res = input_col_as_tuple->getColumns()[index.value()]; /// Wrap into Arrays for (auto it = array_offsets.rbegin(); it != array_offsets.rend(); ++it) res = ColumnArray::create(res, *it); - if (first_arg_is_const) - { + if (input_arg_is_const) res = ColumnConst::create(res, input_rows_count); - } return res; } private: - void recursiveCheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y, size_t depth) const { for (size_t i = 1; i < depth; ++i) @@ -187,22 +160,16 @@ private: void checkArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const { if (isColumnConst(*col_x)) - { checkArrayOffsetsWithFirstArgConst(col_x, col_y); - } else if (isColumnConst(*col_y)) - { checkArrayOffsetsWithFirstArgConst(col_y, col_x); - } else { const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); if (!array_x.hasEqualOffsets(array_y)) - { throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "The argument 1 and argument 3 of function {} have different array sizes", getName()); - } } } @@ -220,23 +187,21 @@ private: size_t row_size = offsets_y.size(); for (size_t row = 0; row < row_size; ++row) { - if (unlikely(offsets_x[0] != offsets_y[row] - prev_offset)) - { + if (offsets_x[0] != offsets_y[row] - prev_offset) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "The argument 1 and argument 3 of function {} have different array sizes", getName()); - } prev_offset = offsets_y[row]; } } - std::optional getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, const size_t argument_size) const + std::optional getElementIndex(const ColumnPtr & index_column, const DataTypeTuple & tuple, size_t argument_size) const { if (checkAndGetColumnConst(index_column.get()) || checkAndGetColumnConst(index_column.get()) || checkAndGetColumnConst(index_column.get()) || checkAndGetColumnConst(index_column.get())) { - size_t index = index_column->getUInt(0); + const size_t index = index_column->getUInt(0); if (index == 0) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indices in tuples are 1-based."); @@ -244,21 +209,20 @@ private: if (index > tuple.getElements().size()) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Index for tuple element is out of range."); - return std::optional(index - 1); + return {index - 1}; } else if (const auto * name_col = checkAndGetColumnConst(index_column.get())) { - auto index = tuple.tryGetPositionByName(name_col->getValue()); - if (index.has_value()) - { - return index; - } + std::optional index = tuple.tryGetPositionByName(name_col->getValue()); - if (argument_size == 2) + if (index.has_value()) + return index; + else { - throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}'", name_col->getValue()); + if (argument_size == 2) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}'", name_col->getValue()); + return std::nullopt; } - return std::nullopt; } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, diff --git a/tests/queries/0_stateless/02116_tuple_element.sql b/tests/queries/0_stateless/02116_tuple_element.sql index c911712684d..bedfedd7c2d 100644 --- a/tests/queries/0_stateless/02116_tuple_element.sql +++ b/tests/queries/0_stateless/02116_tuple_element.sql @@ -14,12 +14,12 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 2) FROM t_tuple_element; SELECT tupleElement(t1, 'a') FROM t_tuple_element; EXPLAIN SYNTAX SELECT tupleElement(t1, 'a') FROM t_tuple_element; -SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError 43 } -SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError 42 } -SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError 10, 47 } -SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError 43 } +SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER } +SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT t2.1 FROM t_tuple_element; EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element; @@ -27,11 +27,11 @@ EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element; SELECT tupleElement(t2, 1) FROM t_tuple_element; EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element; -SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError 42 } -SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 10, 47 } -SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError 43 } +SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER } +SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } DROP TABLE t_tuple_element; diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql index 908a869885b..ba1388cfa57 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.sql +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -10,16 +10,16 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError 127 } -SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError 127 } +SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } DROP TABLE t_tuple_element_default; SELECT '--------------------'; -SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError 645 } -SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError 190 } -SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError 190 } +SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError NUMBER_OF_DIMENSIONS_MISMATCHED } +SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } +SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); EXPLAIN SYNTAX SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); From bf54fb4caad774bb1efe1fd39e7b7abaef1a9869 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 11:06:41 +0000 Subject: [PATCH 41/79] tupleElement(): Return default value for out-of-bounds-index Makes the actual and the documented behavior consistent. --- src/Functions/tupleElement.cpp | 14 ++++++++------ .../02354_tuple_element_with_default.reference | 2 ++ .../02354_tuple_element_with_default.sql | 4 ++-- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index fb8f1d3b48d..8689a095809 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -203,13 +203,15 @@ private: { const size_t index = index_column->getUInt(0); - if (index == 0) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indices in tuples are 1-based."); + if (index > 0 && index <= tuple.getElements().size()) + return {index - 1}; + else + { + if (argument_size == 2) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with index '{}'", index); + return std::nullopt; + } - if (index > tuple.getElements().size()) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Index for tuple element is out of range."); - - return {index - 1}; } else if (const auto * name_col = checkAndGetColumnConst(index_column.get())) { diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.reference b/tests/queries/0_stateless/02354_tuple_element_with_default.reference index d5dfff17ef1..08a1f60e163 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.reference +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.reference @@ -7,6 +7,8 @@ FROM t_tuple_element_default z SELECT tupleElement(t2, \'z\', \'z\') FROM t_tuple_element_default +z +z -------------------- [(3,4)] SELECT tupleElement([(1, 2)], \'a\', [(3, 4)]) diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql index ba1388cfa57..de281c0a868 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.sql +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -10,8 +10,8 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } -SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; +SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; DROP TABLE t_tuple_element_default; From 5b78b3903b468575dc36af84bbb90f673d50a7c0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 12:26:34 +0000 Subject: [PATCH 42/79] Fix logical error in tupleElement() --- src/Functions/tupleElement.cpp | 69 ------------------- .../02286_tuple_numeric_identifier.sql | 4 +- ...02354_tuple_element_with_default.reference | 43 ++++-------- .../02354_tuple_element_with_default.sql | 61 +++++----------- 4 files changed, 34 insertions(+), 143 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 8689a095809..96b5a047419 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -17,11 +17,8 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_INDEX; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NOT_FOUND_COLUMN_IN_BLOCK; - extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; - extern const int SIZES_OF_ARRAYS_DONT_MATCH; } namespace @@ -79,21 +76,7 @@ public: return return_type; } else - { - const IDataType * default_type = arguments[2].type.get(); - size_t default_count_arrays = 0; - - if (const DataTypeArray * default_type_as_array = checkAndGetDataType(default_type)) - default_count_arrays = default_type_as_array->getNumberOfDimensions(); - - if (count_arrays != default_count_arrays) - throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, - "Dimension of types mismatched between first argument and third argument. " - "Dimension of 1st argument: {}. " - "Dimension of 3rd argument: {}", count_arrays, default_count_arrays); - return arguments[2].type; - } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override @@ -128,11 +111,7 @@ public: std::optional index = getElementIndex(arguments[1].column, *input_type_as_tuple, arguments.size()); if (!index.has_value()) - { - if (!array_offsets.empty()) - recursiveCheckArrayOffsets(arguments[0].column, arguments[2].column, array_offsets.size()); return arguments[2].column; - } ColumnPtr res = input_col_as_tuple->getColumns()[index.value()]; @@ -146,54 +125,6 @@ public: } private: - void recursiveCheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y, size_t depth) const - { - for (size_t i = 1; i < depth; ++i) - { - checkArrayOffsets(col_x, col_y); - col_x = assert_cast(col_x.get())->getDataPtr(); - col_y = assert_cast(col_y.get())->getDataPtr(); - } - checkArrayOffsets(col_x, col_y); - } - - void checkArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const - { - if (isColumnConst(*col_x)) - checkArrayOffsetsWithFirstArgConst(col_x, col_y); - else if (isColumnConst(*col_y)) - checkArrayOffsetsWithFirstArgConst(col_y, col_x); - else - { - const auto & array_x = *assert_cast(col_x.get()); - const auto & array_y = *assert_cast(col_y.get()); - if (!array_x.hasEqualOffsets(array_y)) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "The argument 1 and argument 3 of function {} have different array sizes", getName()); - } - } - - void checkArrayOffsetsWithFirstArgConst(ColumnPtr col_x, ColumnPtr col_y) const - { - col_x = assert_cast(col_x.get())->getDataColumnPtr(); - col_y = col_y->convertToFullColumnIfConst(); - const auto & array_x = *assert_cast(col_x.get()); - const auto & array_y = *assert_cast(col_y.get()); - - const auto & offsets_x = array_x.getOffsets(); - const auto & offsets_y = array_y.getOffsets(); - - ColumnArray::Offset prev_offset = 0; - size_t row_size = offsets_y.size(); - for (size_t row = 0; row < row_size; ++row) - { - if (offsets_x[0] != offsets_y[row] - prev_offset) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "The argument 1 and argument 3 of function {} have different array sizes", getName()); - prev_offset = offsets_y[row]; - } - } - std::optional getElementIndex(const ColumnPtr & index_column, const DataTypeTuple & tuple, size_t argument_size) const { if (checkAndGetColumnConst(index_column.get()) diff --git a/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql b/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql index a5fd8e57ad5..f723284ad61 100644 --- a/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql +++ b/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql @@ -12,8 +12,8 @@ SELECT * FROM t_tuple_numeric FORMAT JSONEachRow; SELECT `t`.`1`.`2`, `t`.`1`.`3`, `t`.`4` FROM t_tuple_numeric; SELECT t.1.1, t.1.2, t.2 FROM t_tuple_numeric; -SELECT t.1.3 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX} -SELECT t.4 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX} +SELECT t.1.3 FROM t_tuple_numeric; -- {serverError NOT_FOUND_COLUMN_IN_BLOCK} +SELECT t.4 FROM t_tuple_numeric; -- {serverError NOT_FOUND_COLUMN_IN_BLOCK} SELECT `t`.`1`.`1`, `t`.`1`.`2`, `t`.`2` FROM t_tuple_numeric; -- {serverError UNKNOWN_IDENTIFIER} DROP TABLE t_tuple_numeric; diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.reference b/tests/queries/0_stateless/02354_tuple_element_with_default.reference index 08a1f60e163..499b4c36a10 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.reference +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.reference @@ -1,28 +1,15 @@ -z -SELECT tupleElement(t1, \'z\', \'z\') -FROM t_tuple_element_default -0 -SELECT tupleElement(t1, \'z\', 0) -FROM t_tuple_element_default -z -SELECT tupleElement(t2, \'z\', \'z\') -FROM t_tuple_element_default -z -z --------------------- -[(3,4)] -SELECT tupleElement([(1, 2)], \'a\', [(3, 4)]) --------------------- -SELECT tupleElement(t1, \'a\', [tuple(1)]) -FROM t_tuple_element_default --------------------- -[(0)] -SELECT tupleElement(t1, \'a\', [tuple(0)]) -FROM t_tuple_element_default -[0] -SELECT tupleElement(t1, \'a\', [0]) -FROM t_tuple_element_default -[0] -[0] -SELECT tupleElement(t1, \'a\', [0]) -FROM t_tuple_element_default +hello +world +default +default +[(['a'],1)] +[1,3] +[2,4] +default +-------- +hello +world +default +default +[(['a'],1)] +[[1,2,3]] diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql index de281c0a868..89320f4d210 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.sql +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -1,50 +1,23 @@ -DROP TABLE IF EXISTS t_tuple_element_default; +-- const tuple argument -CREATE TABLE t_tuple_element_default(t1 Tuple(a UInt32, s String), t2 Tuple(UInt32, String)) ENGINE = Memory; -INSERT INTO t_tuple_element_default VALUES ((1, 'a'), (2, 'b')); +SELECT tupleElement(('hello', 'world'), 1, 'default'); +SELECT tupleElement(('hello', 'world'), 2, 'default'); +SELECT tupleElement(('hello', 'world'), 3, 'default'); +SELECT tupleElement(('hello', 'world'), 'xyz', 'default'); +SELECT tupleElement(('hello', 'world'), 3, [([('a')], 1)]); -- arbitrary default value -SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; -SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; +SELECT tupleElement([(1, 2), (3, 4)], 1, 'default'); +SELECT tupleElement([(1, 2), (3, 4)], 2, 'default'); +SELECT tupleElement([(1, 2), (3, 4)], 3, 'default'); -SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; +SELECT '--------'; -DROP TABLE t_tuple_element_default; +-- non-const tuple argument -SELECT '--------------------'; - -SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError NUMBER_OF_DIMENSIONS_MISMATCHED } -SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } -SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } - -SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); -EXPLAIN SYNTAX SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); - -SELECT '--------------------'; - -CREATE TABLE t_tuple_element_default(t1 Array(Tuple(UInt32)), t2 UInt32) ENGINE = Memory; - -SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; - -SELECT '--------------------'; - -INSERT INTO t_tuple_element_default VALUES ([(1)], 100); - -SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; - -SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; - -INSERT INTO t_tuple_element_default VALUES ([(2)], 200); - -SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; - -DROP TABLE t_tuple_element_default; +SELECT tupleElement(materialize(('hello', 'world')), 1, 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 2, 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 3, 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 'xzy', 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 'xzy', [([('a')], 1)]); -- arbitrary default value +SELECT tupleElement([[(count('2147483646'), 1)]], 'aaaa', [[1, 2, 3]]) -- bug #51525 From 8854f05a7534fc844b5a7b00af72ad0ae7c0ac1b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 28 Jun 2023 19:53:49 +0000 Subject: [PATCH 43/79] style fix --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 979e53a72c0..cc88594d84f 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -420,7 +420,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ { if (DB::Exception::enable_job_stack_trace) DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); - + CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); From 7b4e6faece3b172763f3237bf8a42282a289f059 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Jun 2023 18:32:31 +0200 Subject: [PATCH 44/79] Speedup the submodules cloning --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index dab873377ce..be9d569e65c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -151,7 +151,7 @@ function clone_submodules ) git submodule sync - git submodule update --jobs=16 --depth 1 --init "${SUBMODULES_TO_UPDATE[@]}" + git submodule update --jobs=16 --depth 1 --single-branch --init "${SUBMODULES_TO_UPDATE[@]}" git submodule foreach git reset --hard git submodule foreach git checkout @ -f git submodule foreach git clean -xfd From b68d8fa76e7be0ef1ae58ce41c6a326027c3afe6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Jun 2023 18:49:54 +0200 Subject: [PATCH 45/79] Update the OS version for builder and testers docker images --- docker/test/fasttest/Dockerfile | 1 + docker/test/integration/base/Dockerfile | 2 +- docker/test/stateful/Dockerfile | 5 +++-- docker/test/stateless/Dockerfile | 7 ++++--- docker/test/util/Dockerfile | 2 +- 5 files changed, 10 insertions(+), 7 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index ffb13fc774d..da4baa8c687 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -9,6 +9,7 @@ RUN apt-get update \ expect \ file \ lsof \ + odbcinst \ psmisc \ python3 \ python3-lxml \ diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index de8efa20af4..ff50626b6aa 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -49,7 +49,7 @@ RUN arch=${TARGETARCH:-amd64} \ && curl -o mysql-odbc.rpm "https://cdn.mysql.com/archives/mysql-connector-odbc-8.0/mysql-connector-odbc-8.0.27-1.el8.${rarch}.rpm" \ && rpm2archive mysql-odbc.rpm \ && tar xf mysql-odbc.rpm.tgz -C / ./usr/lib64/ \ - && LINK_DIR=$(dpkg -L libodbc1 | rg '^/usr/lib/.*-linux-gnu/odbc$') \ + && LINK_DIR=$(dpkg -L odbc-postgresql | rg '^/usr/lib/.*-linux-gnu/odbc$') \ && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR" \ && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR"/libmyodbc.so diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 71a2e92e3a8..f513735a2d0 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -16,8 +16,9 @@ COPY s3downloader /s3downloader ENV S3_URL="https://clickhouse-datasets.s3.amazonaws.com" ENV DATASETS="hits visits" -RUN npm install -g azurite -RUN npm install tslib +# The following is already done in clickhouse/stateless-test +# RUN npm install -g azurite +# RUN npm install tslib COPY run.sh / CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 40109255a7e..32996140521 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -20,6 +20,7 @@ RUN apt-get update -y \ netcat-openbsd \ nodejs \ npm \ + odbcinst \ openjdk-11-jre-headless \ openssl \ postgresql-client \ @@ -71,7 +72,7 @@ RUN arch=${TARGETARCH:-amd64} \ && chmod +x ./mc ./minio -RUN wget 'https://dlcdn.apache.org/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \ +RUN wget --no-verbose 'https://dlcdn.apache.org/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \ && tar -xvf hadoop-3.3.1.tar.gz \ && rm -rf hadoop-3.3.1.tar.gz @@ -79,8 +80,8 @@ ENV MINIO_ROOT_USER="clickhouse" ENV MINIO_ROOT_PASSWORD="clickhouse" ENV EXPORT_S3_STORAGE_POLICIES=1 -RUN npm install -g azurite -RUN npm install tslib +RUN npm install -g azurite \ + && npm install -g tslib COPY run.sh / COPY setup_minio.sh / diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index a49278e960b..85e888f1df7 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -1,5 +1,5 @@ # docker build -t clickhouse/test-util . -FROM ubuntu:20.04 +FROM ubuntu:22.04 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" From 5a8ce1f2fe5a7048066e9883ec49828e27ab7bd8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Jun 2023 22:26:27 +0200 Subject: [PATCH 46/79] Do not upload the binaries for fast-tests to tests-reports --- docker/test/fasttest/run.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index be9d569e65c..989ed9d2fbb 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -80,7 +80,7 @@ function start_server function clone_root { - git config --global --add safe.directory "$FASTTEST_SOURCE" + [ "$UID" -eq 0 ] && git config --global --add safe.directory "$FASTTEST_SOURCE" git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" ( @@ -202,10 +202,11 @@ function build | ts '%Y-%m-%d %H:%M:%S' \ | tee "$FASTTEST_OUTPUT/test_result.txt" if [ "$COPY_CLICKHOUSE_BINARY_TO_OUTPUT" -eq "1" ]; then - cp programs/clickhouse "$FASTTEST_OUTPUT/clickhouse" + mkdir -p "$FASTTEST_OUTPUT/binaries/" + cp programs/clickhouse "$FASTTEST_OUTPUT/binaries/clickhouse" - strip programs/clickhouse -o "$FASTTEST_OUTPUT/clickhouse-stripped" - zstd --threads=0 "$FASTTEST_OUTPUT/clickhouse-stripped" + strip programs/clickhouse -o programs/clickhouse-stripped + zstd --threads=0 programs/clickhouse-stripped -o "$FASTTEST_OUTPUT/binaries/clickhouse-stripped.zst" fi ccache_status ccache --evict-older-than 1d ||: From edcf981c4837468bf4d08dd86183b38c120913c2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Jun 2023 13:11:11 +0200 Subject: [PATCH 47/79] Update mysql odbc connector --- docker/test/integration/base/Dockerfile | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index ff50626b6aa..270b40e23a6 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -46,12 +46,13 @@ RUN arch=${TARGETARCH:-amd64} \ arm64) rarch=aarch64 ;; \ esac \ && cd /tmp \ - && curl -o mysql-odbc.rpm "https://cdn.mysql.com/archives/mysql-connector-odbc-8.0/mysql-connector-odbc-8.0.27-1.el8.${rarch}.rpm" \ + && curl -o mysql-odbc.rpm "https://cdn.mysql.com/archives/mysql-connector-odbc-8.0/mysql-connector-odbc-8.0.32-1.el9.${rarch}.rpm" \ && rpm2archive mysql-odbc.rpm \ && tar xf mysql-odbc.rpm.tgz -C / ./usr/lib64/ \ - && LINK_DIR=$(dpkg -L odbc-postgresql | rg '^/usr/lib/.*-linux-gnu/odbc$') \ - && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR" \ - && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR"/libmyodbc.so + && rm mysql-odbc.rpm mysql-odbc.rpm.tgz \ + && ODBC_DIR=$(dpkg -L odbc-postgresql | rg '^/usr/lib/.*-linux-gnu/odbc$') \ + && ln -s /usr/lib64/libmyodbc8a.so "$ODBC_DIR" \ + && ln -s /usr/lib64/libmyodbc8a.so "$ODBC_DIR"/libmyodbc.so # Unfortunately this is required for a single test for conversion data from zookeeper to clickhouse-keeper. # ZooKeeper is not started by default, but consumes some space in containers. From ead81879b43d8b86f9b73f97fc0ac4277a79f047 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 21 Jun 2023 16:33:43 +0200 Subject: [PATCH 48/79] Update integration tests runner too --- .../integration/helper_container/Dockerfile | 5 +- docker/test/integration/runner/Dockerfile | 50 ++++++++++--------- tests/integration/helpers/cluster.py | 2 - tests/integration/helpers/network.py | 4 -- tests/integration/pytest.ini | 3 ++ tests/integration/runner | 46 +++++++---------- tests/integration/test_storage_hudi/test.py | 2 +- 7 files changed, 51 insertions(+), 61 deletions(-) diff --git a/docker/test/integration/helper_container/Dockerfile b/docker/test/integration/helper_container/Dockerfile index 6a093081bf2..60adaea1796 100644 --- a/docker/test/integration/helper_container/Dockerfile +++ b/docker/test/integration/helper_container/Dockerfile @@ -2,4 +2,7 @@ # Helper docker container to run iptables without sudo FROM alpine -RUN apk add -U iproute2 +RUN apk add --no-cache -U iproute2 \ + && for bin in iptables iptables-restore iptables-save; \ + do ln -sf xtables-nft-multi "/sbin/$bin"; \ + done diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 14c97e479f6..40627354f70 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -1,5 +1,5 @@ # docker build -t clickhouse/integration-tests-runner . -FROM ubuntu:20.04 +FROM ubuntu:22.04 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" @@ -56,17 +56,19 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ /var/lib/apt/lists/* \ /var/cache/debconf \ /tmp/* \ - && apt-get clean + && apt-get clean \ + && dockerd --version; docker --version -RUN dockerd --version; docker --version RUN python3 -m pip install --no-cache-dir \ PyMySQL \ - aerospike==4.0.0 \ - avro==1.10.2 \ + aerospike==11.1.0 \ asyncio \ + avro==1.10.2 \ + azure-storage-blob \ cassandra-driver \ - confluent-kafka==1.5.0 \ + confluent-kafka==1.9.2 \ + delta-spark==2.3.0 \ dict2xml \ dicttoxml \ docker \ @@ -76,47 +78,47 @@ RUN python3 -m pip install --no-cache-dir \ kafka-python \ kazoo \ lz4 \ + meilisearch==0.18.3 \ minio \ nats-py \ protobuf \ - psycopg2-binary==2.8.6 \ + psycopg2-binary==2.9.6 \ + pyhdfs \ pymongo==3.11.0 \ + pyspark==3.3.2 \ pytest \ pytest-order==1.0.0 \ - pytest-timeout \ pytest-random \ - pytest-xdist \ pytest-repeat \ + pytest-timeout \ + pytest-xdist \ pytz \ redis \ - tzlocal==2.1 \ - urllib3 \ requests-kerberos \ - pyspark==3.3.2 \ - delta-spark==2.2.0 \ - pyhdfs \ - azure-storage-blob \ - meilisearch==0.18.3 - -COPY modprobe.sh /usr/local/bin/modprobe -COPY dockerd-entrypoint.sh /usr/local/bin/ -COPY compose/ /compose/ -COPY misc/ /misc/ + tzlocal==2.1 \ + urllib3 +# Hudi supports only spark 3.3.*, not 3.4 RUN curl -fsSL -O https://dlcdn.apache.org/spark/spark-3.3.2/spark-3.3.2-bin-hadoop3.tgz \ && tar xzvf spark-3.3.2-bin-hadoop3.tgz -C / \ && rm spark-3.3.2-bin-hadoop3.tgz # download spark and packages # if you change packages, don't forget to update them in tests/integration/helpers/cluster.py -RUN echo ":quit" | /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" > /dev/null +RUN echo ":quit" | /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.3.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" > /dev/null RUN set -x \ && addgroup --system dockremap \ - && adduser --system dockremap \ + && adduser --system dockremap \ && adduser dockremap dockremap \ && echo 'dockremap:165536:65536' >> /etc/subuid \ - && echo 'dockremap:165536:65536' >> /etc/subgid + && echo 'dockremap:165536:65536' >> /etc/subgid + +COPY modprobe.sh /usr/local/bin/modprobe +COPY dockerd-entrypoint.sh /usr/local/bin/ +COPY compose/ /compose/ +COPY misc/ /misc/ + # Same options as in test/base/Dockerfile # (in case you need to override them in tests) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c77e67062a1..967eaaa78a5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3054,7 +3054,6 @@ CLICKHOUSE_STAY_ALIVE_COMMAND = "bash -c \"trap 'pkill tail' INT TERM; {} --daem CLICKHOUSE_START_COMMAND ) -# /run/xtables.lock passed inside for correct iptables --wait DOCKER_COMPOSE_TEMPLATE = """ version: '2.3' services: @@ -3066,7 +3065,6 @@ services: - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ - /etc/passwd:/etc/passwd:ro - - /run/xtables.lock:/run/xtables.lock:ro {binary_volume} {odbc_bridge_volume} {library_bridge_volume} diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 4859a8c5946..2df560708e0 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -271,10 +271,6 @@ class _NetworkManager: image_name, auto_remove=True, command=("sleep %s" % self.container_exit_timeout), - # /run/xtables.lock passed inside for correct iptables --wait - volumes={ - "/run/xtables.lock": {"bind": "/run/xtables.lock", "mode": "ro"} - }, detach=True, network_mode="host", ) diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index 772c96f7361..e40959bd37b 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -19,3 +19,6 @@ markers = long_run: marks tests which run for a long time addopts = -m 'not long_run' +; 'The asyncore module is deprecated' comes from casandra driver +filterwarnings = + ignore:The asyncore module is deprecated:DeprecationWarning diff --git a/tests/integration/runner b/tests/integration/runner index 301a707a78d..df52f587eee 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -395,39 +395,27 @@ if __name__ == "__main__": if args.keyword_expression: args.pytest_args += ["-k", args.keyword_expression] - cmd_base = "docker run {net} {tty} --rm --name {name} --privileged \ - --volume={odbc_bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ - --volume={library_bridge_bin}:/clickhouse-library-bridge \ - --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ - --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume=/run:/run/host:ro \ - {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ - -e XTABLES_LOCKFILE=/run/host/xtables.lock \ - -e PYTHONUNBUFFERED=1 \ - {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} {rand} -vvv' {img}".format( - net=net, - tty=tty, - bin=args.binary, - odbc_bridge_bin=args.odbc_bridge_binary, - library_bridge_bin=args.library_bridge_binary, - base_cfg=args.base_configs_dir, - cases_dir=args.cases_dir, - src_dir=args.src_dir, - env_tags=env_tags, - env_cleanup=env_cleanup, - parallel=parallel_args, - rand=rand_args, - opts=" ".join(args.pytest_args).replace("'", "\\'"), - tests_list=" ".join(args.tests_list), - dockerd_internal_volume=dockerd_internal_volume, - img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, - name=CONTAINER_NAME, + pytest_opts = " ".join(args.pytest_args).replace("'", "\\'") + tests_list = " ".join(args.tests_list) + cmd_base = ( + f"docker run {net} {tty} --rm --name {CONTAINER_NAME} " + "--privileged --dns-search='.' " # since recent dns search leaks from host + f"--volume={args.odbc_bridge_binary}:/clickhouse-odbc-bridge " + f"--volume={args.binary}:/clickhouse " + f"--volume={args.library_bridge_binary}:/clickhouse-library-bridge " + f"--volume={args.base_configs_dir}:/clickhouse-config " + f"--volume={args.cases_dir}:/ClickHouse/tests/integration " + f"--volume={args.src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos " + f"--volume=/run:/run/host:ro {dockerd_internal_volume} {env_tags} {env_cleanup} " + "-e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 -e PYTHONUNBUFFERED=1 " + f"-e PYTEST_OPTS='{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv'" + f" {DIND_INTEGRATION_TESTS_IMAGE_NAME}:{args.docker_image_version}" ) cmd = cmd_base + " " + args.command cmd_pre_pull = ( - cmd_base - + " find /compose -name docker_compose_*.yml -exec docker-compose -f '{}' pull \;" + f"{cmd_base} find /compose -name docker_compose_*.yml " + r"-exec docker-compose -f '{}' pull \;" ) containers = subprocess.check_output( diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index de9cde43609..2b77f4d6d61 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -79,7 +79,7 @@ def run_query(instance, query, stdin=None, settings=None): def write_hudi_from_df(spark, table_name, df, result_path, mode="overwrite"): - if mode is "overwrite": + if mode == "overwrite": hudi_write_mode = "insert_overwrite" else: hudi_write_mode = "upsert" From 953f1c78855500bdd2a6a603749a65082e0a4067 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 02:36:30 +0200 Subject: [PATCH 49/79] Upload fast test binaries to builds bucket --- tests/ci/fast_test_check.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 2849759a3ee..d5198e5c3d7 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -11,6 +11,7 @@ from typing import List, Tuple from github import Github +from build_check import get_release_or_pr from clickhouse_helper import ( ClickHouseHelper, mark_flaky_tests, @@ -31,6 +32,7 @@ from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results +from version_helper import get_version_from_repo NAME = "Fast test" @@ -189,6 +191,17 @@ def main(): ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, NAME, test_results) + s3_path_prefix = os.path.join( + get_release_or_pr(pr_info, get_version_from_repo())[0], + pr_info.sha, + "fast_tests", + ) + build_urls = s3_helper.upload_build_folder_to_s3( + os.path.join(output_path, "binaries"), + s3_path_prefix, + keep_dirs_in_s3_path=False, + upload_symlinks=False, + ) report_url = upload_results( s3_helper, @@ -197,6 +210,7 @@ def main(): test_results, [run_log_path] + additional_logs, NAME, + build_urls, ) print(f"::notice ::Report url: {report_url}") post_commit_status(commit, state, report_url, description, NAME, pr_info) From 91dc6dfe34f1239b936506324d552f578cf39fa3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 18:20:23 +0200 Subject: [PATCH 50/79] Update mysql-php-client to the recent rolling version --- docker/test/integration/mysql_php_client/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/mysql_php_client/Dockerfile b/docker/test/integration/mysql_php_client/Dockerfile index 55db4d15a7f..0e11ae023e6 100644 --- a/docker/test/integration/mysql_php_client/Dockerfile +++ b/docker/test/integration/mysql_php_client/Dockerfile @@ -1,7 +1,7 @@ # docker build -t clickhouse/mysql-php-client . # MySQL PHP client docker container -FROM php:8.0.18-cli +FROM php:8-cli-alpine COPY ./client.crt client.crt COPY ./client.key client.key From b2bfe2eb87e30e217839ec9b6c063a2e234c6f1f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 19:10:44 +0200 Subject: [PATCH 51/79] The error is changed in psycopg2=2.9.6 --- tests/integration/test_postgresql_protocol/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index e1d8cbf9bcc..de01bba6862 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -111,7 +111,7 @@ def test_psql_client(started_cluster): def test_python_client(started_cluster): node = cluster.instances["node"] - with pytest.raises(py_psql.InternalError) as exc_info: + with pytest.raises(py_psql.OperationalError) as exc_info: ch = py_psql.connect( host=node.ip_address, port=server_port, @@ -122,9 +122,7 @@ def test_python_client(started_cluster): cur = ch.cursor() cur.execute("select name from tables;") - assert exc_info.value.args == ( - "Query execution failed.\nDB::Exception: Table default.tables doesn't exist\nSSL connection has been closed unexpectedly\n", - ) + assert exc_info.value.args == ("SSL connection has been closed unexpectedly\n",) ch = py_psql.connect( host=node.ip_address, From 22c8f1c0be46f5cc5115d6e8c55a963fa466b12e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 01:46:29 +0200 Subject: [PATCH 52/79] Fix test_jemalloc_percpu_arena after upgrade --- tests/integration/test_jemalloc_percpu_arena/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_jemalloc_percpu_arena/test.py b/tests/integration/test_jemalloc_percpu_arena/test.py index 0dccde3776e..8de3dcd7ea2 100755 --- a/tests/integration/test_jemalloc_percpu_arena/test.py +++ b/tests/integration/test_jemalloc_percpu_arena/test.py @@ -26,7 +26,7 @@ def run_command_in_container(cmd, *args): "run", "--rm", *args, - "ubuntu:20.04", + "ubuntu:22.04", "sh", "-c", cmd, From 406896f9a95658a27f298c4c8b154467f8d5cbb8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:10:47 +0200 Subject: [PATCH 53/79] Create cgroupsv2 subtree for docker-in-docker --- docker/test/integration/runner/dockerd-entrypoint.sh | 11 +++++++++++ tests/integration/test_cgroup_limit/test.py | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 347d904d5c0..3c4ff522b36 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -12,6 +12,17 @@ echo '{ "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"] }' | dd of=/etc/docker/daemon.json 2>/dev/null +if [ -f /sys/fs/cgroup/cgroup.controllers ]; then + # move the processes from the root group to the /init group, + # otherwise writing subtree_control fails with EBUSY. + # An error during moving non-existent process (i.e., "cat") is ignored. + mkdir -p /sys/fs/cgroup/init + xargs -rn1 < /sys/fs/cgroup/cgroup.procs > /sys/fs/cgroup/init/cgroup.procs || : + # enable controllers + sed -e 's/ / +/g' -e 's/^/+/' < /sys/fs/cgroup/cgroup.controllers \ + > /sys/fs/cgroup/cgroup.subtree_control +fi + # In case of test hung it is convenient to use pytest --pdb to debug it, # and on hung you can simply press Ctrl-C and it will spawn a python pdb, # but on SIGINT dockerd will exit, so ignore it to preserve the daemon. diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index f6392eca4d7..4ab12436dc9 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -22,7 +22,7 @@ def run_command_in_container(cmd, *args): "run", "--rm", *args, - "ubuntu:20.04", + "ubuntu:22.04", "sh", "-c", cmd, From a18300c497a051762f1ffcb5f366d50591a812f1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:28:01 +0200 Subject: [PATCH 54/79] Improve logging in integration/test_cgroup_limit --- tests/integration/test_cgroup_limit/test.py | 26 +++++++++++---------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index 4ab12436dc9..e77b0f70960 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 +import logging import os import math import subprocess @@ -16,18 +17,19 @@ def run_command_in_container(cmd, *args): f"{alternative_binary}:/usr/bin/clickhouse", ) - return subprocess.check_output( - [ - "docker", - "run", - "--rm", - *args, - "ubuntu:22.04", - "sh", - "-c", - cmd, - ] - ) + command = [ + "docker", + "run", + "--rm", + *args, + "ubuntu:22.04", + "sh", + "-c", + cmd, + ] + + logging.debug("Command: %s", " ".join(command)) + return subprocess.check_output(command) def run_with_cpu_limit(cmd, num_cpus, *args): From 4f1982e25e9313442f312feb70bc25bd88dfae2c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:51:18 +0200 Subject: [PATCH 55/79] API of pika for rabbitmq has changed --- tests/integration/test_storage_rabbitmq/test.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index b4dcf86e0ba..943e7742018 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -858,7 +858,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): if len(insert_messages) == 50: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -921,7 +921,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): if len(insert_messages) == 50: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -2991,7 +2991,7 @@ def test_format_with_prefix_and_suffix(rabbitmq_cluster): if len(insert_messages) == 2: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3050,7 +3050,7 @@ def test_max_rows_per_message(rabbitmq_cluster): if len(insert_messages) == 2: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3148,7 +3148,7 @@ def test_row_based_formats(rabbitmq_cluster): if insert_messages == 2: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3211,7 +3211,7 @@ def test_block_based_formats_1(rabbitmq_cluster): if len(insert_messages) == 3: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3296,7 +3296,7 @@ def test_block_based_formats_2(rabbitmq_cluster): if insert_messages == 9: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() From f85460d8ef6e5996f9ec56296b3a155344772dd8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:56:06 +0200 Subject: [PATCH 56/79] Enable tests with broken channel.start_consuming --- tests/integration/test_storage_rabbitmq/test.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 943e7742018..14764dd3835 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2953,7 +2953,6 @@ def test_rabbitmq_address(rabbitmq_cluster): instance2.query("drop table rabbit_out sync") -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_format_with_prefix_and_suffix(rabbitmq_cluster): instance.query( """ @@ -3002,7 +3001,6 @@ def test_format_with_prefix_and_suffix(rabbitmq_cluster): ) -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_max_rows_per_message(rabbitmq_cluster): num_rows = 5 @@ -3075,7 +3073,6 @@ def test_max_rows_per_message(rabbitmq_cluster): assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_row_based_formats(rabbitmq_cluster): num_rows = 10 @@ -3172,7 +3169,6 @@ def test_row_based_formats(rabbitmq_cluster): assert result == expected -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_block_based_formats_1(rabbitmq_cluster): instance.query( """ @@ -3234,7 +3230,6 @@ def test_block_based_formats_1(rabbitmq_cluster): ] -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_block_based_formats_2(rabbitmq_cluster): num_rows = 100 From 0680f0988c906a01567fae54b2097f5ad60f749d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 03:02:03 +0200 Subject: [PATCH 57/79] Enable other working tests --- tests/integration/test_storage_rabbitmq/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 14764dd3835..751279f5e5a 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1020,7 +1020,6 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): ), "ClickHouse lost some messages: {}".format(result) -@pytest.mark.skip(reason="Flaky") def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query( """ @@ -2050,7 +2049,6 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ) -@pytest.mark.skip(reason="Timeout: FIXME") def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): logging.getLogger("pika").propagate = False instance.query( From f728f9735048d5cea37bb422fefc72ffcab649ce Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 13:54:01 +0200 Subject: [PATCH 58/79] Add cgroupsv2 for getCGroupLimitedCPUCores --- src/Common/getNumberOfPhysicalCPUCores.cpp | 86 +++++++++++++++++++++- 1 file changed, 83 insertions(+), 3 deletions(-) diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index ed82c59140d..8fbb32e911f 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -1,4 +1,5 @@ #include "getNumberOfPhysicalCPUCores.h" +#include #include "config.h" #if defined(OS_LINUX) @@ -7,6 +8,8 @@ #endif #include +#include +#include #include #include @@ -15,7 +18,7 @@ namespace { #if defined(OS_LINUX) -int32_t readFrom(const char * filename, int default_value) +int32_t readFrom(const std::filesystem::path & filename, int default_value) { std::ifstream infile(filename); if (!infile.is_open()) @@ -31,10 +34,87 @@ int32_t readFrom(const char * filename, int default_value) uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count) { uint32_t quota_count = default_cpu_count; + std::filesystem::path prefix = "/sys/fs/cgroup"; + /// cgroupsv2 + std::ifstream contr_file(prefix / "cgroup.controllers"); + if (contr_file.is_open()) + { + /// First, we identify the cgroup the process belongs + std::ifstream cgroup_name_file("/proc/self/cgroup"); + if (!cgroup_name_file.is_open()) + return default_cpu_count; + + // cgroup_name_file always starts with '0::/' for v2 + cgroup_name_file.ignore(4); + std::string cgroup_name; + cgroup_name_file >> cgroup_name; + + std::filesystem::path current_cgroup; + if (cgroup_name.empty()) + current_cgroup = prefix; + else + current_cgroup = prefix / cgroup_name; + + // Looking for cpu.max in directories from the current cgroup to the top level + // It does not stop on the first time since the child could have a greater value than parent + while (current_cgroup != prefix.parent_path()) + { + std::ifstream cpu_max_file(current_cgroup / "cpu.max"); + current_cgroup = current_cgroup.parent_path(); + if (cpu_max_file.is_open()) + { + std::string cpu_limit_str; + float cpu_period; + cpu_max_file >> cpu_limit_str >> cpu_period; + if (cpu_limit_str != "max" && cpu_period != 0) + { + float cpu_limit = std::stof(cpu_limit_str); + quota_count = std::min(static_cast(ceil(cpu_limit / cpu_period)), quota_count); + } + } + } + current_cgroup = prefix / cgroup_name; + // Looking for cpuset.cpus.effective in directories from the current cgroup to the top level + while (current_cgroup != prefix.parent_path()) + { + std::ifstream cpuset_cpus_file(current_cgroup / "cpuset.cpus.effective"); + current_cgroup = current_cgroup.parent_path(); + if (cpuset_cpus_file.is_open()) + { + // The line in the file is "0,2-4,6,9-14" cpu numbers + // It's always grouped and ordered + std::vector cpu_ranges; + std::string cpuset_line; + cpuset_cpus_file >> cpuset_line; + if (cpuset_line.empty()) + continue; + boost::split(cpu_ranges, cpuset_line, boost::is_any_of(",")); + uint32_t cpus_count = 0; + for (const std::string& cpu_number_or_range : cpu_ranges) + { + std::vector cpu_range; + boost::split(cpu_range, cpu_number_or_range, boost::is_any_of("-")); + + if (cpu_range.size() == 2) + { + int start = std::stoi(cpu_range[0]); + int end = std::stoi(cpu_range[1]); + cpus_count += (end - start) + 1; + } + else + cpus_count++; + } + quota_count = std::min(cpus_count, quota_count); + break; + } + } + return quota_count; + } + /// cgroupsv1 /// Return the number of milliseconds per period process is guaranteed to run. /// -1 for no quota - int cgroup_quota = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", -1); - int cgroup_period = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_period_us", -1); + int cgroup_quota = readFrom(prefix / "cpu/cpu.cfs_quota_us", -1); + int cgroup_period = readFrom(prefix / "cpu/cpu.cfs_period_us", -1); if (cgroup_quota > -1 && cgroup_period > 0) quota_count = static_cast(ceil(static_cast(cgroup_quota) / static_cast(cgroup_period))); From 45232770e06079b2b346209ae7b412597ff5996f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 00:41:23 +0300 Subject: [PATCH 59/79] Try to fix deadlock in ZooKeeper client --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 14b31c6a411..dd555e39529 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -362,6 +362,16 @@ ZooKeeper::ZooKeeper( { tryLogCurrentException(log, "Failed to connect to ZooKeeper"); + try + { + requests_queue.finish(); + socket.shutdown(); + } + catch (...) + { + tryLogCurrentException(log); + } + send_thread.join(); receive_thread.join(); From 37ad50bd637b8bfd6612e0b80b8f6dd388a19595 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 28 Jun 2023 19:40:40 -0300 Subject: [PATCH 60/79] Update settings.md --- .../server-configuration-parameters/settings.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 40c1b8d64a1..bad7e388377 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2120,7 +2120,13 @@ This section contains the following parameters: - `operation_timeout_ms` — Maximum timeout for one operation in milliseconds. - `root` — The [znode](http://zookeeper.apache.org/doc/r3.5.5/zookeeperOver.html#Nodes+and+ephemeral+nodes) that is used as the root for znodes used by the ClickHouse server. Optional. - `identity` — User and password, that can be required by ZooKeeper to give access to requested znodes. Optional. - +- zookeeper_load_balancing - Specifies the algorithm of ZooKeeper node selection. + * random - randomly selects one of ZooKeeper nodes. + * in_order - selects the first ZooKeeper node, if it's not available then the second, and so on. + * nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the server’s hostname. + * first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes. + * round_robin - selects the first ZooKeeper node, if reconnection happens selects the next. + **Example configuration** ``` xml @@ -2139,6 +2145,8 @@ This section contains the following parameters: /path/to/zookeeper/node user:password + + random ``` From 340262814a721ca7be1523ff1194c404b0f03de8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 29 Jun 2023 12:24:19 +0200 Subject: [PATCH 61/79] Avoid additional maven requests for spark tests --- docker/test/integration/runner/Dockerfile | 6 +++++- tests/integration/helpers/cluster.py | 10 ++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 40627354f70..38d8ed5f223 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -105,7 +105,11 @@ RUN curl -fsSL -O https://dlcdn.apache.org/spark/spark-3.3.2/spark-3.3.2-bin-had # download spark and packages # if you change packages, don't forget to update them in tests/integration/helpers/cluster.py -RUN echo ":quit" | /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.3.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" > /dev/null +RUN packages="org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,\ +io.delta:delta-core_2.12:2.3.0,\ +org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" \ + && /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "$packages" > /dev/null \ + && find /root/.ivy2/ -name '*.jar' -exec ln -sf {} /spark-3.3.2-bin-hadoop3/jars/ \; RUN set -x \ && addgroup --system dockremap \ diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 967eaaa78a5..21398790be3 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -624,10 +624,12 @@ class ClickHouseCluster: # if you change packages, don't forget to update them in docker/test/integration/runner/dockerd-entrypoint.sh ( pyspark.sql.SparkSession.builder.appName("spark_test") - .config( - "spark.jars.packages", - "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0", - ) + # The jars are now linked to "$SPARK_HOME/jars" and we don't + # need packages to be downloaded once and once again + # .config( + # "spark.jars.packages", + # "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0", + # ) .master("local") .getOrCreate() .stop() From 1d9d712984df7d63bbcaf232b73dde562619ba55 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 29 Jun 2023 13:32:17 +0200 Subject: [PATCH 62/79] use timeout instead trap in 01443_merge_truncate_long.sh --- .../0_stateless/01443_merge_truncate_long.sh | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01443_merge_truncate_long.sh b/tests/queries/0_stateless/01443_merge_truncate_long.sh index ce867d32c1a..65b9bcd366e 100755 --- a/tests/queries/0_stateless/01443_merge_truncate_long.sh +++ b/tests/queries/0_stateless/01443_merge_truncate_long.sh @@ -13,20 +13,20 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY ()" -function thread() +function thread_optimize() { - trap 'BREAK=1' 2 - - while [[ -z "${BREAK}" ]] + while true; do ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL;" 2>&1 | tr -d '\n' | rg -v 'Cancelled merging parts' ||: done } -thread & -pid=$! +TIMEOUT=15 +export -f thread_optimize +timeout $TIMEOUT bash -c thread_optimize 2> /dev/null & -for i in {1..100}; do +for i in {1..100}; +do echo " INSERT INTO t VALUES (0); INSERT INTO t VALUES (0); @@ -36,7 +36,6 @@ for i in {1..100}; do " done | ${CLICKHOUSE_CLIENT} --multiquery -kill -2 "$pid" wait $CLICKHOUSE_CLIENT -q "DROP TABLE t" From b56d1602e841663f073c6f53a06e848b89c9214e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Jun 2023 14:25:13 +0200 Subject: [PATCH 63/79] Invalidate opened file cache when removing file --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/IO/OpenedFileCache.h | 9 +++++++-- src/Interpreters/Cache/FileSegment.h | 2 ++ src/Interpreters/Cache/Metadata.cpp | 5 +++++ 4 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 76d54f9d27c..e4dadb4eb21 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -160,7 +160,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm if (use_external_buffer) local_read_settings.local_fs_buffer_size = 0; - auto buf = createReadBufferFromFileBase(path, local_read_settings); + auto buf = createReadBufferFromFileBase(path, local_read_settings, std::nullopt, std::nullopt, file_segment.getFlagsForLocalRead()); if (getFileSizeFromReadBuffer(*buf) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read from an empty cache file: {}", path); diff --git a/src/IO/OpenedFileCache.h b/src/IO/OpenedFileCache.h index 844e5b31d11..61e502a494b 100644 --- a/src/IO/OpenedFileCache.h +++ b/src/IO/OpenedFileCache.h @@ -72,6 +72,13 @@ public: return res; } + void remove(const std::string & path, int flags) + { + Key key(path, flags); + std::lock_guard lock(mutex); + files.erase(key); + } + static OpenedFileCache & instance() { static OpenedFileCache res; @@ -82,5 +89,3 @@ public: using OpenedFileCachePtr = std::shared_ptr; } - - diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 681c0d719e4..b8fa9486472 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -161,6 +161,8 @@ public: String getPathInLocalCache() const; + int getFlagsForLocalRead() const { return O_RDONLY | O_CLOEXEC; } + /** * ========== Methods for _any_ file segment's owner ======================== */ diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f799bae1e10..f201455384b 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -406,6 +406,11 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm if (exists) { fs::remove(path); + + int flags = file_segment->getFlagsForLocalRead(); + OpenedFileCache::instance().remove(path, flags); + OpenedFileCache::instance().remove(path, flags | O_DIRECT); + LOG_TEST(key_metadata->log, "Removed file segment at path: {}", path); } else if (file_segment->downloaded_size) From 2a4cfeb01bcff1112d6615ddff529286ff024186 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 29 Jun 2023 15:27:40 +0200 Subject: [PATCH 64/79] Add comment --- src/Interpreters/Cache/Metadata.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f201455384b..d4389e0d0b8 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -407,7 +407,10 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm { fs::remove(path); + /// Clear OpenedFileCache to avoid reading from incorrect file descriptor. int flags = file_segment->getFlagsForLocalRead(); + /// Files are created with flags from file_segment->getFlagsForLocalRead() + /// plus optionally O_DIRECT is added, depends on query setting, so remove both. OpenedFileCache::instance().remove(path, flags); OpenedFileCache::instance().remove(path, flags | O_DIRECT); From d1bb8f24c0f99ad4af37747306e06691b34c8955 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 29 Jun 2023 08:55:06 +0000 Subject: [PATCH 65/79] Retry chroot creation in ZK --- docker/test/stateless/run.sh | 27 ++++++++++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 21cb3168083..914e51a9f66 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -90,6 +90,30 @@ sleep 5 attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 +function run_with_retry() +{ + set +e + + local total_retries="$1" + shift + + local retry=0 + + until [ "$retry" -ge "$total_retries" ] + do + if "$@"; then + set -e + return + else + retry=$((retry + 1)) + sleep 3 + fi + done + + echo "Command '$*' failed after $total_retries retries, exiting" + exit 1 +} + function run_tests() { set -x @@ -138,7 +162,8 @@ function run_tests() ADDITIONAL_OPTIONS+=('--report-logs-stats') clickhouse-test "00001_select_1" > /dev/null ||: - clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" ||: + + run_with_retry 5 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ From a28728b7e537f26faf0c4785fc9fb96218178594 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 17:40:10 +0300 Subject: [PATCH 66/79] Update DatabaseReplicatedWorker.cpp (cherry picked from commit 43fc1af1b3855652536037cc2b5111f5ae1ac983) --- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 593d0655777..4976f54e417 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -161,7 +161,7 @@ bool DatabaseReplicatedDDLWorker::waitForReplicaToProcessAllEntries(UInt64 timeo LOG_TRACE(log, "Waiting for worker thread to process all entries before {}, current task is {}", max_log, current_task); bool processed = wait_current_task_change.wait_for(lock, std::chrono::milliseconds(timeout_ms), [&]() { - return zookeeper->expired() || current_task == max_log || stop_flag; + return zookeeper->expired() || current_task >= max_log || stop_flag; }); if (!processed) From f48de18640467e6302c1bd48799e386d4aa39437 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Jun 2023 20:59:01 +0000 Subject: [PATCH 67/79] Update version_date.tsv and changelogs after v23.4.5.22-stable --- docs/changelogs/v23.4.5.22-stable.md | 27 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 29 insertions(+) create mode 100644 docs/changelogs/v23.4.5.22-stable.md diff --git a/docs/changelogs/v23.4.5.22-stable.md b/docs/changelogs/v23.4.5.22-stable.md new file mode 100644 index 00000000000..2d61f5b11cf --- /dev/null +++ b/docs/changelogs/v23.4.5.22-stable.md @@ -0,0 +1,27 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.4.5.22-stable (0ced5d6a8da) FIXME as compared to v23.4.4.16-stable (747ba4fc6a0) + +#### Build/Testing/Packaging Improvement +* Backported in [#51530](https://github.com/ClickHouse/ClickHouse/issues/51530): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#51570](https://github.com/ClickHouse/ClickHouse/issues/51570): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix broken index analysis when binary operator contains a null constant argument [#50177](https://github.com/ClickHouse/ClickHouse/pull/50177) ([Amos Bird](https://github.com/amosbird)). +* Fix reconnecting of HTTPS session when target host IP was changed [#50240](https://github.com/ClickHouse/ClickHouse/pull/50240) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix incorrect constant folding [#50536](https://github.com/ClickHouse/ClickHouse/pull/50536) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). +* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Increase max array size in group bitmap [#50620](https://github.com/ClickHouse/ClickHouse/pull/50620) ([Kruglov Pavel](https://github.com/Avogar)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 5c8dd0d2481..4ca5d1d7497 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,10 +2,12 @@ v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 +v23.4.5.22-stable 2023-06-29 v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.7.5-lts 2023-06-29 v23.3.6.7-lts 2023-06-28 v23.3.5.9-lts 2023-06-22 v23.3.4.17-lts 2023-06-17 From a0bf1708837667c1f3d00dded295ec5ade7a4ac9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Jun 2023 21:01:19 +0000 Subject: [PATCH 68/79] Update version_date.tsv and changelogs after v23.3.7.5-lts --- docs/changelogs/v23.3.7.5-lts.md | 16 ++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 18 insertions(+) create mode 100644 docs/changelogs/v23.3.7.5-lts.md diff --git a/docs/changelogs/v23.3.7.5-lts.md b/docs/changelogs/v23.3.7.5-lts.md new file mode 100644 index 00000000000..7a5fd5a19b6 --- /dev/null +++ b/docs/changelogs/v23.3.7.5-lts.md @@ -0,0 +1,16 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.7.5-lts (bc683c11c92) FIXME as compared to v23.3.6.7-lts (7e3f0a271b7) + +#### Build/Testing/Packaging Improvement +* Backported in [#51568](https://github.com/ClickHouse/ClickHouse/issues/51568): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 5c8dd0d2481..4ca5d1d7497 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,10 +2,12 @@ v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 +v23.4.5.22-stable 2023-06-29 v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.7.5-lts 2023-06-29 v23.3.6.7-lts 2023-06-28 v23.3.5.9-lts 2023-06-22 v23.3.4.17-lts 2023-06-17 From fd5f6ee2f99989f9ab5c80ecbce1a0b6ab7ae109 Mon Sep 17 00:00:00 2001 From: Ramazan Polat Date: Fri, 30 Jun 2023 04:32:41 +0300 Subject: [PATCH 69/79] Update parts.md Fix misalignment of `active` --- docs/en/operations/system-tables/parts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index 9159d1e9284..5829e5ad313 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -27,7 +27,7 @@ Columns: Data storing format is controlled by the `min_bytes_for_wide_part` and `min_rows_for_wide_part` settings of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table. - - `active` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that indicates whether the data part is active. If a data part is active, it’s used in a table. Otherwise, it’s deleted. Inactive data parts remain after merging. +- `active` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that indicates whether the data part is active. If a data part is active, it’s used in a table. Otherwise, it’s deleted. Inactive data parts remain after merging. - `marks` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` by the index granularity (usually 8192) (this hint does not work for adaptive granularity). From 710911f7dc18ecaa5a400d4efe6587a9b656d937 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Jun 2023 11:39:08 +0300 Subject: [PATCH 70/79] Update CHANGELOG.md --- CHANGELOG.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c1e0dba4465..bf6b309ef2c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,7 +18,6 @@ * CGroups metrics related to CPU are replaced with one metric, `CGroupMaxCPU` for better usability. The `Normalized` CPU usage metrics will be normalized to CGroups limits instead of the total number of CPUs when they are set. This closes [#50836](https://github.com/ClickHouse/ClickHouse/issues/50836). [#50835](https://github.com/ClickHouse/ClickHouse/pull/50835) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* 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)). * The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Added option `--rename_files_after_processing `. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)). * Add support for `TRUNCATE` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)). @@ -26,7 +25,7 @@ * Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)). * Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)). * Clickhouse-client can now be called with a connection string instead of "--host", "--port", "--user" etc. [#50689](https://github.com/ClickHouse/ClickHouse/pull/50689) ([Alexey Gerasimchuck](https://github.com/Demilivor)). -* Add setting `session_timezone`, it is used as default timezone for session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)). +* Add setting `session_timezone`; it is used as the default timezone for a session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)). * Codec DEFLATE_QPL is now controlled via server setting "enable_deflate_qpl_codec" (default: false) instead of setting "allow_experimental_codecs". This marks DEFLATE_QPL non-experimental. [#50775](https://github.com/ClickHouse/ClickHouse/pull/50775) ([Robert Schulze](https://github.com/rschu1ze)). #### Performance Improvement From 2b55734ccf533a4fd9180b57672bc35823ed66d0 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 30 Jun 2023 02:52:03 -0700 Subject: [PATCH 71/79] Fix MergeTreeMarksLoader segfaulting if marks file is longer than expected (#51636) Co-authored-by: Nikita Mikhaylov --- .../MergeTree/MergeTreeMarksLoader.cpp | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 9a5576f0ad2..5c722eec380 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -135,6 +135,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() if (!index_granularity_info.mark_type.adaptive) { /// Read directly to marks. + chassert(expected_uncompressed_size == plain_marks.size() * sizeof(MarkInCompressedFile)); reader->readStrict(reinterpret_cast(plain_marks.data()), expected_uncompressed_size); if (!reader->eof()) @@ -148,23 +149,25 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() } else { - size_t i = 0; - size_t granularity; - while (!reader->eof()) + for (size_t i = 0; i < marks_count; ++i) { + if (reader->eof()) + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Cannot read all marks from file {}, marks expected {} (bytes size {}), marks read {} (bytes size {})", + mrk_path, marks_count, expected_uncompressed_size, i, reader->count()); + + size_t granularity; reader->readStrict( reinterpret_cast(plain_marks.data() + i * columns_in_mark), columns_in_mark * sizeof(MarkInCompressedFile)); readIntBinary(granularity, *reader); - ++i; } - if (i * mark_size != expected_uncompressed_size) - { + if (!reader->eof()) throw Exception( ErrorCodes::CANNOT_READ_ALL_DATA, - "Cannot read all marks from file {}, marks expected {} (bytes size {}), marks read {} (bytes size {})", - mrk_path, marks_count, expected_uncompressed_size, i, reader->count()); - } + "Too many marks in file {}, marks expected {} (bytes size {})", + mrk_path, marks_count, expected_uncompressed_size); } auto res = std::make_shared(plain_marks); From 84872517708ad03c764e616fd28d5d1d12593697 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 30 Jun 2023 14:46:50 +0200 Subject: [PATCH 72/79] Update File Cache disk configuration settings default values --- docs/en/operations/storing-data.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 5804ad8545b..3f4456d7e12 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -188,9 +188,9 @@ These settings should be defined in the disk configuration section. - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. -- `max_file_segment_size` - a maximum size of a single cache file in bytes or in readable format (`ki, Mi, Gi, etc`, example `10Gi`). Default: `104857600` (`100Mi`). +- `max_file_segment_size` - a maximum size of a single cache file in bytes or in readable format (`ki, Mi, Gi, etc`, example `10Gi`). Default: `8388608` (`8Mi`). -- `max_elements` - a limit for a number of cache files. Default: `1048576`. +- `max_elements` - a limit for a number of cache files. Default: `10000000`. File Cache **query/profile settings**: From bf06e18c348d6d8bd23d4184d4bf3eee89ed4835 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 30 Jun 2023 09:03:25 -0400 Subject: [PATCH 73/79] docs clickhouse-static-files-uploader and demo repo --- .../en/engines/table-engines/mergetree-family/mergetree.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index a1c2fbdbe50..67043ef1062 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -949,7 +949,14 @@ The example uses `type=web`, but any disk type can be configured as dynamic, eve #### Example dynamic web storage +:::tip +A [demo dataset](https://github.com/ClickHouse/web-tables-demo) is hosted in GitHub. To prepare your own tables for web storage see the tool [clickhouse-static-files-uploader](/docs/en/operations/storing-data.md/#storing-data-on-webserver) +::: + +In this `ATTACH TABLE` query the `UUID` provided matches the directory name of the data, and the endpoint is the URL for the raw GitHub content. + ```sql +# highlight-next-line ATTACH TABLE uk_price_paid UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7' ( price UInt32, From 904c533a84e5766b4e5ca5e886d7a826cefdd0d4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 30 Jun 2023 09:32:54 -0400 Subject: [PATCH 74/79] spelling list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f25d082e5a6..2802e52c288 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2459,6 +2459,7 @@ unrounded untracked untrusted untuple +uploader uploaders upperUTF uptime From 91579453895b7dc88b0aec78bdde6311a6fc5a39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 30 Jun 2023 15:43:04 +0200 Subject: [PATCH 75/79] Add missing settings --- docs/en/operations/storing-data.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 3f4456d7e12..fd82c955c40 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -184,7 +184,9 @@ These settings should be defined in the disk configuration section. - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. -- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default: `0`, e.g. the data is cached at the first attempt to read it. +- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached, defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. + +- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. From 0b98406bd345f54c386435a11d126c07234f1aef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 30 Jun 2023 16:13:43 +0200 Subject: [PATCH 76/79] Add default values to False --- docs/en/operations/storing-data.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index fd82c955c40..17d62673a8e 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -184,9 +184,9 @@ These settings should be defined in the disk configuration section. - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. -- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached, defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. +- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default :`false`. This threshold can be defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. -- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). +- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold. Default :`false`. This threshold can be defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. From 689ff6f996585da91cf80eb2916f1d23706c1f30 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 30 Jun 2023 16:15:26 +0200 Subject: [PATCH 77/79] Update storing-data.md --- docs/en/operations/storing-data.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 17d62673a8e..fe6e8e15b0c 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -184,9 +184,9 @@ These settings should be defined in the disk configuration section. - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. -- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default :`false`. This threshold can be defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. +- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default: `false`. This threshold can be defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. -- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold. Default :`false`. This threshold can be defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). +- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold. Default: `false`. This threshold can be defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. From 5ee7f47b03d3b741db80dae38dfb169f338d9dcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Jun 2023 17:09:56 +0200 Subject: [PATCH 78/79] Update autogenerated version to 23.7.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 +++++----- .../System/StorageSystemContributors.generated.cpp | 9 +++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 015037b2de6..821b7b46855 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 54475) +SET(VERSION_REVISION 54476) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 6) +SET(VERSION_MINOR 7) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 2fec796e73efda10a538a03af3205ce8ffa1b2de) -SET(VERSION_DESCRIBE v23.6.1.1-testing) -SET(VERSION_STRING 23.6.1.1) +SET(VERSION_GITHASH d1c7e13d08868cb04d3562dcced704dd577cb1df) +SET(VERSION_DESCRIBE v23.7.1.1-testing) +SET(VERSION_STRING 23.7.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index f83ee3197fe..f84c554afc0 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -226,6 +226,7 @@ const char * auto_contributors[] { "Carbyn", "Carlos Rodríguez Hernández", "Caspian", + "Chang Chen", "Chao Ma", "Chao Wang", "CheSema", @@ -291,6 +292,7 @@ const char * auto_contributors[] { "Dmitry Belyavtsev", "Dmitry Bilunov", "Dmitry Galuza", + "Dmitry Kardymon", "Dmitry Krylov", "Dmitry Luhtionov", "Dmitry Moskowski", @@ -408,6 +410,7 @@ const char * auto_contributors[] { "HeenaBansal2009", "Hiroaki Nakamura", "Hongbin", + "Hongbin Ma", "Hosun Lee", "HuFuwang", "Hui Wang", @@ -491,6 +494,7 @@ const char * auto_contributors[] { "Josh Taylor", "João Figueiredo", "Julian Gilyadov", + "Julian Maicher", "Julian Zhou", "Julio Jimenez", "Junfu Wu", @@ -917,6 +921,7 @@ const char * auto_contributors[] { "Thom O'Connor", "Thomas Berdy", "Thomas Casteleyn", + "Thomas Panetti", "Tian Xinhui", "Tiaonmmn", "Tigran Khudaverdyan", @@ -1022,6 +1027,7 @@ const char * auto_contributors[] { "Y Lu", "Yakko Majuri", "Yakov Olkhovskiy", + "YalalovSM", "Yangkuan Liu", "Yatian Xu", "Yatsishin Ilya", @@ -1300,6 +1306,7 @@ const char * auto_contributors[] { "kashwy", "keenwolf", "kevin wan", + "kevinyhzou", "kgurjev", "khamadiev", "kigerzhang", @@ -1545,6 +1552,7 @@ const char * auto_contributors[] { "tiger.yan", "tison", "topvisor", + "tpanetti", "turbo jason", "tyrionhuang", "ubuntu", @@ -1673,6 +1681,7 @@ const char * auto_contributors[] { "董海镔", "谢磊", "贾顺名(Jarvis)", + "郭小龙", "陈小玉", "靳阳", "黄朝晖", From 8ed1ec49d181123227192e7dfac6ee0119a2e8d7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 30 Jun 2023 15:21:13 +0000 Subject: [PATCH 79/79] Update version_date.tsv and changelogs after v23.6.1.1524-stable --- SECURITY.md | 1 + docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.6.1.1524-stable.md | 301 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 306 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.6.1.1524-stable.md diff --git a/SECURITY.md b/SECURITY.md index 1864eb6e9e5..4ba5f13d09c 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,6 +13,7 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.6 | ✔️ | | 23.5 | ✔️ | | 23.4 | ✔️ | | 23.3 | ✔️ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 6d53a6f4c51..f13fcdc14d6 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.5.4.25" +ARG VERSION="23.6.1.1524" 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 91b22346f13..5e5be3f6d73 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.5.4.25" +ARG VERSION="23.6.1.1524" 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 0ed0e4e1168..8693193455f 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.5.4.25" +ARG VERSION="23.6.1.1524" 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.6.1.1524-stable.md b/docs/changelogs/v23.6.1.1524-stable.md new file mode 100644 index 00000000000..6d295d61ef4 --- /dev/null +++ b/docs/changelogs/v23.6.1.1524-stable.md @@ -0,0 +1,301 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.6.1.1524-stable (d1c7e13d088) FIXME as compared to v23.5.1.3174-stable (2fec796e73e) + +#### Backward Incompatible Change +* Delete feature `do_not_evict_index_and_mark_files` in the fs cache. This feature was only making things worse. [#51253](https://github.com/ClickHouse/ClickHouse/pull/51253) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove ALTER support for experimental LIVE VIEW. [#51287](https://github.com/ClickHouse/ClickHouse/pull/51287) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Add setting `session_timezone`, it is used as default timezone for session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)). +* Added overlay database engine and representation of a directory as a database This commit adds 4 databases: 1. DatabaseOverlay: Implements the IDatabase interface. Allow to combine multiple databases, such as FileSystem and Memory. Internally, it stores a vector with other database pointers and proxies requests to them in turn until it is executed successfully. 2. DatabaseFilesystem: allows to read-only interact with files stored on the file system. Internally, it uses TableFunctionFile to implicitly load file when a user requests the table. Result of TableFunctionFile call cached inside to provide quick access. 3. DatabaseS3: allows to read-only interact with s3 storage. It uses TableFunctionS3 to implicitly load table from s3 4. DatabaseHDFS: allows to interact with hdfs storage. It uses TableFunctionHDFS to implicitly load table from hdfs. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). +* Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)). +* Added option `--rename_files_after_processing `. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)). +* 1. Add `TableFunctionRedis` 3. Add table engine Redis 4. Add `RedisCommon` which contains Redis related tools and types 5. Support `equals` and `in` filter push down into Redis. [#50150](https://github.com/ClickHouse/ClickHouse/pull/50150) ([JackyWoo](https://github.com/JackyWoo)). +* Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)). +* Clickhouse-client can now be called with a connection instead of "--host", "--port", "--user" etc. [#50689](https://github.com/ClickHouse/ClickHouse/pull/50689) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Codec DEFLATE_QPL is now controlled via server setting "enable_deflate_qpl_codec" (default: false) instead of setting "allow_experimental_codecs". This marks QPL_DEFLATE non-experimental. [#50775](https://github.com/ClickHouse/ClickHouse/pull/50775) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Performance Improvement +* Improve performance with enabled QueryProfiler using thread-local timer_id instead of global object. [#48778](https://github.com/ClickHouse/ClickHouse/pull/48778) ([Jiebin Sun](https://github.com/jiebinn)). +* Rewrite CapnProto input/output format to improve its performance. Map column names and CapnProto fields case insensitive, fix reading/writing of nested structure fields. [#49752](https://github.com/ClickHouse/ClickHouse/pull/49752) ([Kruglov Pavel](https://github.com/Avogar)). +* Optimize parquet write performance for parallel threads. [#50102](https://github.com/ClickHouse/ClickHouse/pull/50102) ([Hongbin Ma](https://github.com/binmahone)). +* ### Documentation entry for user-facing changes Disable `parallelize_output_from_storages` for processing MATERIALIZED VIEWs and storages with one block only. [#50214](https://github.com/ClickHouse/ClickHouse/pull/50214) ([Azat Khuzhin](https://github.com/azat)). +* Merge PR https://github.com/ClickHouse/ClickHouse/pull/46558 (Avoid processing already sorted data). Avoid block permutation during sort if the block is already sorted. [#50697](https://github.com/ClickHouse/ClickHouse/pull/50697) ([Maksim Kita](https://github.com/kitaisreal)). +* In the earlier PRs ([#50062](https://github.com/ClickHouse/ClickHouse/issues/50062), [#50307](https://github.com/ClickHouse/ClickHouse/issues/50307)), we used to propose an optimization pattern which transforms the predicates with toYear/toYYYYMM into its equivalent but converter-free form. This transformation could bring significant performance impact to some workloads, such as SSB. However, as issue [#50628](https://github.com/ClickHouse/ClickHouse/issues/50628) indicated, these two PRs would introduce some issues which may results in incomplete query results, and as a result, they were reverted by [#50629](https://github.com/ClickHouse/ClickHouse/issues/50629). [#50951](https://github.com/ClickHouse/ClickHouse/pull/50951) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Make multiple list requests to ZooKeeper in parallel to speed up reading from system.zookeeper table. [#51042](https://github.com/ClickHouse/ClickHouse/pull/51042) ([Alexander Gololobov](https://github.com/davenger)). +* Speedup initialization of DateTime lookup tables for time zones. This should reduce startup/connect time of clickhouse client especially in debug build as it is rather heavy. [#51347](https://github.com/ClickHouse/ClickHouse/pull/51347) ([Alexander Gololobov](https://github.com/davenger)). + +#### Improvement +* Allow to cast IPv6 to IPv4 address for CIDR ::ffff:0:0/96 (IPv4-mapped addresses). [#49759](https://github.com/ClickHouse/ClickHouse/pull/49759) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update MongoDB protocol to support MongoDB 5.1 version and newer. Support for the versions with the old protocol (<3.6) is preserved. Closes [#45621](https://github.com/ClickHouse/ClickHouse/issues/45621), [#49879](https://github.com/ClickHouse/ClickHouse/issues/49879). [#50061](https://github.com/ClickHouse/ClickHouse/pull/50061) ([Nikolay Degterinsky](https://github.com/evillique)). +* Improved scheduling of merge selecting and cleanup tasks in `ReplicatedMergeTree`. The tasks will not be executed too frequently when there's nothing to merge or cleanup. Added settings `max_merge_selecting_sleep_ms`, `merge_selecting_sleep_slowdown_factor`, `max_cleanup_delay_period` and `cleanup_thread_preferred_points_per_iteration`. It should close [#31919](https://github.com/ClickHouse/ClickHouse/issues/31919). [#50107](https://github.com/ClickHouse/ClickHouse/pull/50107) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support parallel replicas with the analyzer. [#50441](https://github.com/ClickHouse/ClickHouse/pull/50441) ([Raúl Marín](https://github.com/Algunenano)). +* Add setting `input_format_max_bytes_to_read_for_schema_inference` to limit the number of bytes to read in schema inference. Closes [#50577](https://github.com/ClickHouse/ClickHouse/issues/50577). [#50592](https://github.com/ClickHouse/ClickHouse/pull/50592) ([Kruglov Pavel](https://github.com/Avogar)). +* Respect setting input_format_as_default in schema inference. [#50602](https://github.com/ClickHouse/ClickHouse/pull/50602) ([Kruglov Pavel](https://github.com/Avogar)). +* Make filter push down through cross join. [#50605](https://github.com/ClickHouse/ClickHouse/pull/50605) ([Han Fei](https://github.com/hanfei1991)). +* Actual lz4 version is used now. [#50621](https://github.com/ClickHouse/ClickHouse/pull/50621) ([Nikita Taranov](https://github.com/nickitat)). +* Allow to skip trailing empty lines in CSV/TSV/CustomSeparated formats via settings `input_format_csv_skip_trailing_empty_lines`, `input_format_tsv_skip_trailing_empty_lines` and `input_format_custom_skip_trailing_empty_lines` (disabled by default). Closes [#49315](https://github.com/ClickHouse/ClickHouse/issues/49315). [#50635](https://github.com/ClickHouse/ClickHouse/pull/50635) ([Kruglov Pavel](https://github.com/Avogar)). +* Functions "toDateOrDefault|OrNull()" and "accuateCast[OrDefault|OrNull]()" now correctly parse numeric arguments. [#50709](https://github.com/ClickHouse/ClickHouse/pull/50709) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Currently, the csv input format can not parse the csv file with whitespace or \t field delimiter, and these delimiters is supported in spark. [#50712](https://github.com/ClickHouse/ClickHouse/pull/50712) ([KevinyhZou](https://github.com/KevinyhZou)). +* Settings `number_of_mutations_to_delay` and `number_of_mutations_to_throw` are enabled by default now with values 500 and 1000 respectively. [#50726](https://github.com/ClickHouse/ClickHouse/pull/50726) ([Anton Popov](https://github.com/CurtizJ)). +* Keeper improvement: add feature flags for Keeper API. Each feature flag can be disabled or enabled by defining it under `keeper_server.feature_flags` config. E.g. to enable `CheckNotExists` request, `keeper_server.feature_flags.check_not_exists` should be set to `1` on Keeper. [#50796](https://github.com/ClickHouse/ClickHouse/pull/50796) ([Antonio Andelic](https://github.com/antonio2368)). +* The dashboard correctly shows missing values. This closes [#50831](https://github.com/ClickHouse/ClickHouse/issues/50831). [#50832](https://github.com/ClickHouse/ClickHouse/pull/50832) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CGroups metrics related to CPU are replaced with one metric, `CGroupMaxCPU` for better usability. The `Normalized` CPU usage metrics will be normalized to CGroups limits instead of the total number of CPUs when they are set. This closes [#50836](https://github.com/ClickHouse/ClickHouse/issues/50836). [#50835](https://github.com/ClickHouse/ClickHouse/pull/50835) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Relax the thresholds for "too many parts" to be more modern. Return the backpressure during long-running insert queries. [#50856](https://github.com/ClickHouse/ClickHouse/pull/50856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Added the possibility to use date and time arguments in syslog timestamp format in functions parseDateTimeBestEffort*() and parseDateTime64BestEffort*(). [#50925](https://github.com/ClickHouse/ClickHouse/pull/50925) ([Victor Krasnov](https://github.com/sirvickr)). +* Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)). +* Add embedded keeper-client to standalone keeper binary. [#50964](https://github.com/ClickHouse/ClickHouse/pull/50964) ([pufit](https://github.com/pufit)). +* Command line parameter "--password" in clickhouse-client can now be specified only once. [#50966](https://github.com/ClickHouse/ClickHouse/pull/50966) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix data lakes slowness because of synchronous head requests. (Related to Iceberg/Deltalake/Hudi being slow with a lot of files). [#50976](https://github.com/ClickHouse/ClickHouse/pull/50976) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use `hash_of_all_files` from `system.parts` to check identity of parts during on-cluster backups. [#50997](https://github.com/ClickHouse/ClickHouse/pull/50997) ([Vitaly Baranov](https://github.com/vitlibar)). +* The system table zookeeper_connection connected_time identifies the time when the connection is established (standard format), and session_uptime_elapsed_seconds is added, which labels the duration of the established connection session (in seconds). [#51026](https://github.com/ClickHouse/ClickHouse/pull/51026) ([郭小龙](https://github.com/guoxiaolongzte)). +* Show halves of checksums in `system.parts`, `system.projection_parts` and in error messages in the correct order. [#51040](https://github.com/ClickHouse/ClickHouse/pull/51040) ([Vitaly Baranov](https://github.com/vitlibar)). +* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)). +* Add total_bytes_to_read to Progress packet in TCP protocol for better Progress bar. [#51158](https://github.com/ClickHouse/ClickHouse/pull/51158) ([Kruglov Pavel](https://github.com/Avogar)). +* Better checking of data parts on disks with filesystem cache. [#51164](https://github.com/ClickHouse/ClickHouse/pull/51164) ([Anton Popov](https://github.com/CurtizJ)). +* Disable cache setting `do_not_evict_index_and_mark_files` (Was enabled in `23.5`). [#51222](https://github.com/ClickHouse/ClickHouse/pull/51222) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix sometimes not correct current_elements_num in fs cache. [#51242](https://github.com/ClickHouse/ClickHouse/pull/51242) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add random sleep before merges/mutations execution to split load more evenly between replicas in case of zero-copy replication. [#51282](https://github.com/ClickHouse/ClickHouse/pull/51282) ([alesapin](https://github.com/alesapin)). +* The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* We have found a bug in LLVM that makes the usage of `compile_expressions` setting unsafe. It is disabled by default. [#51368](https://github.com/ClickHouse/ClickHouse/pull/51368) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Issue [#50220](https://github.com/ClickHouse/ClickHouse/issues/50220) reports a core in `grace_hash` join. We finally reproduce the exception on local, and found that the issue is related to the failure of creating temporary file. Somehow this is triggered in https://github.com/ClickHouse/ClickHouse/pull/49816 https://github.com/ClickHouse/ClickHouse/pull/49483. [#51382](https://github.com/ClickHouse/ClickHouse/pull/51382) ([lgbo](https://github.com/lgbo-ustc)). + +#### Build/Testing/Packaging Improvement +* Update contrib/re2 to 2023-06-02. [#50949](https://github.com/ClickHouse/ClickHouse/pull/50949) ([Yuriy Chernyshov](https://github.com/georgthegreat)). +* ClickHouse server will print the list of changed settings on fatal errors. This closes [#51137](https://github.com/ClickHouse/ClickHouse/issues/51137). [#51138](https://github.com/ClickHouse/ClickHouse/pull/51138) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In https://github.com/ClickHouse/ClickHouse/pull/51143 the fasstests failed, but the status wasn't created because of the chown `file not found`. This addresses it. Decrease the default values for `http-max-field-value-size` and `http_max_field_name_size` to 128K. [#51163](https://github.com/ClickHouse/ClickHouse/pull/51163) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update Ubuntu version in docker containers. [#51180](https://github.com/ClickHouse/ClickHouse/pull/51180) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Allow building ClickHouse with clang-17. [#51300](https://github.com/ClickHouse/ClickHouse/pull/51300) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [SQLancer](https://github.com/sqlancer/sqlancer) check is considered stable as bugs that were triggered by it are fixed. Now failures of SQLancer check will be reported as failed check status. [#51340](https://github.com/ClickHouse/ClickHouse/pull/51340) ([Ilya Yatsishin](https://github.com/qoega)). +* Making our CI even better. [#51494](https://github.com/ClickHouse/ClickHouse/pull/51494) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Report loading status for executable dictionaries correctly [#48775](https://github.com/ClickHouse/ClickHouse/pull/48775) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Proper mutation of skip indices and projections [#50104](https://github.com/ClickHouse/ClickHouse/pull/50104) ([Amos Bird](https://github.com/amosbird)). +* Cleanup moving parts [#50489](https://github.com/ClickHouse/ClickHouse/pull/50489) ([vdimir](https://github.com/vdimir)). +* Fix backward compatibility for IP types hashing in aggregate functions [#50551](https://github.com/ClickHouse/ClickHouse/pull/50551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix Log family table return wrong rows count after truncate [#50585](https://github.com/ClickHouse/ClickHouse/pull/50585) ([flynn](https://github.com/ucasfl)). +* Fix bug in `uniqExact` parallel merging [#50590](https://github.com/ClickHouse/ClickHouse/pull/50590) ([Nikita Taranov](https://github.com/nickitat)). +* Revert recent grace hash join changes [#50699](https://github.com/ClickHouse/ClickHouse/pull/50699) ([vdimir](https://github.com/vdimir)). +* Query Cache: Try to fix bad cast from ColumnConst to ColumnVector [#50704](https://github.com/ClickHouse/ClickHouse/pull/50704) ([Robert Schulze](https://github.com/rschu1ze)). +* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid storing logs in Keeper containing unknown operation [#50751](https://github.com/ClickHouse/ClickHouse/pull/50751) ([Antonio Andelic](https://github.com/antonio2368)). +* SummingMergeTree support for DateTime64 [#50797](https://github.com/ClickHouse/ClickHouse/pull/50797) ([Jordi Villar](https://github.com/jrdi)). +* Add compat setting for non-const timezones [#50834](https://github.com/ClickHouse/ClickHouse/pull/50834) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). +* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix checking the lock file too often while writing a backup [#50889](https://github.com/ClickHouse/ClickHouse/pull/50889) ([Vitaly Baranov](https://github.com/vitlibar)). +* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix race azure blob storage iterator [#50936](https://github.com/ClickHouse/ClickHouse/pull/50936) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix erroneous `sort_description` propagation in `CreatingSets` [#50955](https://github.com/ClickHouse/ClickHouse/pull/50955) ([Nikita Taranov](https://github.com/nickitat)). +* Fix iceberg V2 optional metadata parsing [#50974](https://github.com/ClickHouse/ClickHouse/pull/50974) ([Kseniia Sumarokova](https://github.com/kssenii)). +* MaterializedMySQL: Keep parentheses for empty table overrides [#50977](https://github.com/ClickHouse/ClickHouse/pull/50977) ([Val Doroshchuk](https://github.com/valbok)). +* Fix crash in BackupCoordinationStageSync::setError() [#51012](https://github.com/ClickHouse/ClickHouse/pull/51012) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix subtly broken copy-on-write of ColumnLowCardinality dictionary [#51064](https://github.com/ClickHouse/ClickHouse/pull/51064) ([Michael Kolupaev](https://github.com/al13n321)). +* Generate safe IVs [#51086](https://github.com/ClickHouse/ClickHouse/pull/51086) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Fix ineffective query cache for SELECTs with subqueries [#51132](https://github.com/ClickHouse/ClickHouse/pull/51132) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix Set index with constant nullable comparison. [#51205](https://github.com/ClickHouse/ClickHouse/pull/51205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a crash in s3 and s3Cluster functions [#51209](https://github.com/ClickHouse/ClickHouse/pull/51209) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix core dump when compile expression [#51231](https://github.com/ClickHouse/ClickHouse/pull/51231) ([LiuNeng](https://github.com/liuneng1994)). +* Fix use-after-free in StorageURL when switching URLs [#51260](https://github.com/ClickHouse/ClickHouse/pull/51260) ([Michael Kolupaev](https://github.com/al13n321)). +* Updated check for parameterized view [#51272](https://github.com/ClickHouse/ClickHouse/pull/51272) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix multiple writing of same file to backup [#51299](https://github.com/ClickHouse/ClickHouse/pull/51299) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove garbage from function `transform` [#51350](https://github.com/ClickHouse/ClickHouse/pull/51350) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix MSan report in lowerUTF8/upperUTF8 [#51371](https://github.com/ClickHouse/ClickHouse/pull/51371) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* fs cache: fix a bit incorrect use_count after [#44985](https://github.com/ClickHouse/ClickHouse/issues/44985) [#51406](https://github.com/ClickHouse/ClickHouse/pull/51406) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix segfault in MathUnary [#51499](https://github.com/ClickHouse/ClickHouse/pull/51499) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix logical assert in `tupleElement()` with default values [#51534](https://github.com/ClickHouse/ClickHouse/pull/51534) ([Robert Schulze](https://github.com/rschu1ze)). +* fs cache: remove file from opened file cache immediately when evicting file [#51596](https://github.com/ClickHouse/ClickHouse/pull/51596) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Deprecate delete-on-destroy.txt [#49181](https://github.com/ClickHouse/ClickHouse/pull/49181) ([Alexander Gololobov](https://github.com/davenger)). +* Attempt to increase the general runners' survival rate [#49283](https://github.com/ClickHouse/ClickHouse/pull/49283) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Refactor subqueries for IN [#49570](https://github.com/ClickHouse/ClickHouse/pull/49570) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Test plan optimization analyzer [#50095](https://github.com/ClickHouse/ClickHouse/pull/50095) ([Igor Nikonov](https://github.com/devcrafter)). +* Implement endianness-independent serialization for quantileTiming [#50324](https://github.com/ClickHouse/ClickHouse/pull/50324) ([ltrk2](https://github.com/ltrk2)). +* require `finalize()` call before d-tor for all writes buffers [#50395](https://github.com/ClickHouse/ClickHouse/pull/50395) ([Sema Checherinda](https://github.com/CheSema)). +* Implement big-endian support for the deterministic reservoir sampler [#50405](https://github.com/ClickHouse/ClickHouse/pull/50405) ([ltrk2](https://github.com/ltrk2)). +* Fix compilation error on big-endian platforms [#50406](https://github.com/ClickHouse/ClickHouse/pull/50406) ([ltrk2](https://github.com/ltrk2)). +* Attach gdb in stateless tests [#50487](https://github.com/ClickHouse/ClickHouse/pull/50487) ([Kruglov Pavel](https://github.com/Avogar)). +* JIT infrastructure refactoring [#50531](https://github.com/ClickHouse/ClickHouse/pull/50531) ([Maksim Kita](https://github.com/kitaisreal)). +* Analyzer: Do not apply Query Tree optimizations on shards [#50584](https://github.com/ClickHouse/ClickHouse/pull/50584) ([Dmitry Novik](https://github.com/novikd)). +* Increase max array size in group bitmap [#50620](https://github.com/ClickHouse/ClickHouse/pull/50620) ([Kruglov Pavel](https://github.com/Avogar)). +* Misc Annoy index improvements [#50661](https://github.com/ClickHouse/ClickHouse/pull/50661) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix reading negative decimals in avro format [#50668](https://github.com/ClickHouse/ClickHouse/pull/50668) ([Kruglov Pavel](https://github.com/Avogar)). +* Unify priorities for connection pools [#50675](https://github.com/ClickHouse/ClickHouse/pull/50675) ([Sergei Trifonov](https://github.com/serxa)). +* Prostpone check of outdated parts [#50676](https://github.com/ClickHouse/ClickHouse/pull/50676) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Unify priorities: `IExecutableTask`s [#50677](https://github.com/ClickHouse/ClickHouse/pull/50677) ([Sergei Trifonov](https://github.com/serxa)). +* Disable grace_hash join in stress tests [#50693](https://github.com/ClickHouse/ClickHouse/pull/50693) ([vdimir](https://github.com/vdimir)). +* ReverseTransform small improvement [#50698](https://github.com/ClickHouse/ClickHouse/pull/50698) ([Maksim Kita](https://github.com/kitaisreal)). +* Support OPTIMIZE for temporary tables [#50710](https://github.com/ClickHouse/ClickHouse/pull/50710) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Refactor reading from object storages [#50711](https://github.com/ClickHouse/ClickHouse/pull/50711) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in log message of cached buffer [#50723](https://github.com/ClickHouse/ClickHouse/pull/50723) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new keywords into projections documentation [#50743](https://github.com/ClickHouse/ClickHouse/pull/50743) ([YalalovSM](https://github.com/YalalovSM)). +* Fix build for aarch64 (temporary disable azure) [#50770](https://github.com/ClickHouse/ClickHouse/pull/50770) ([alesapin](https://github.com/alesapin)). +* Update version after release [#50772](https://github.com/ClickHouse/ClickHouse/pull/50772) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v23.5.1.3174-stable [#50774](https://github.com/ClickHouse/ClickHouse/pull/50774) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update CHANGELOG.md [#50788](https://github.com/ClickHouse/ClickHouse/pull/50788) ([Ilya Yatsishin](https://github.com/qoega)). +* Update version_date.tsv and changelogs after v23.2.7.32-stable [#50809](https://github.com/ClickHouse/ClickHouse/pull/50809) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Desctructing --> Destructing [#50810](https://github.com/ClickHouse/ClickHouse/pull/50810) ([Robert Schulze](https://github.com/rschu1ze)). +* Don't mark a part as broken on `Poco::TimeoutException` [#50811](https://github.com/ClickHouse/ClickHouse/pull/50811) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Rename azure_blob_storage to azureBlobStorage [#50812](https://github.com/ClickHouse/ClickHouse/pull/50812) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix ParallelReadBuffer seek [#50820](https://github.com/ClickHouse/ClickHouse/pull/50820) ([Michael Kolupaev](https://github.com/al13n321)). +* [RFC] Print git hash when crashing [#50823](https://github.com/ClickHouse/ClickHouse/pull/50823) ([Michael Kolupaev](https://github.com/al13n321)). +* Add tests for function "transform" [#50833](https://github.com/ClickHouse/ClickHouse/pull/50833) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v23.5.2.7-stable [#50844](https://github.com/ClickHouse/ClickHouse/pull/50844) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Updated changelog with azureBlobStorage table function & engine entry [#50850](https://github.com/ClickHouse/ClickHouse/pull/50850) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update easy_tasks_sorted_ru.md [#50853](https://github.com/ClickHouse/ClickHouse/pull/50853) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Document x86 / ARM prerequisites for Docker image [#50867](https://github.com/ClickHouse/ClickHouse/pull/50867) ([Robert Schulze](https://github.com/rschu1ze)). +* MaterializedMySQL: Add test_named_collections [#50874](https://github.com/ClickHouse/ClickHouse/pull/50874) ([Val Doroshchuk](https://github.com/valbok)). +* Update version_date.tsv and changelogs after v22.8.18.31-lts [#50881](https://github.com/ClickHouse/ClickHouse/pull/50881) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.3.3.52-lts [#50882](https://github.com/ClickHouse/ClickHouse/pull/50882) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.4.3.48-stable [#50883](https://github.com/ClickHouse/ClickHouse/pull/50883) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* MaterializedMySQL: Add additional test case to insert_with_modify_binlog_checksum [#50884](https://github.com/ClickHouse/ClickHouse/pull/50884) ([Val Doroshchuk](https://github.com/valbok)). +* Update broken tests list [#50886](https://github.com/ClickHouse/ClickHouse/pull/50886) ([Dmitry Novik](https://github.com/novikd)). +* Fix LOGICAL_ERROR in snowflakeToDateTime*() [#50893](https://github.com/ClickHouse/ClickHouse/pull/50893) ([Robert Schulze](https://github.com/rschu1ze)). +* Tests with parallel replicas are no more "always green" [#50896](https://github.com/ClickHouse/ClickHouse/pull/50896) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Slightly more information in error message about cached disk [#50897](https://github.com/ClickHouse/ClickHouse/pull/50897) ([Michael Kolupaev](https://github.com/al13n321)). +* do not call finalize after exception [#50907](https://github.com/ClickHouse/ClickHouse/pull/50907) ([Sema Checherinda](https://github.com/CheSema)). +* Update Annoy docs [#50912](https://github.com/ClickHouse/ClickHouse/pull/50912) ([Robert Schulze](https://github.com/rschu1ze)). +* A bit safer UserDefinedSQLFunctionVisitor [#50913](https://github.com/ClickHouse/ClickHouse/pull/50913) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update contribe/orc in .gitmodules [#50920](https://github.com/ClickHouse/ClickHouse/pull/50920) ([San](https://github.com/santrancisco)). +* MaterializedMySQL: Add missing DROP DATABASE for tests [#50924](https://github.com/ClickHouse/ClickHouse/pull/50924) ([Val Doroshchuk](https://github.com/valbok)). +* Fix 'Illegal column timezone' in stress tests [#50929](https://github.com/ClickHouse/ClickHouse/pull/50929) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix tests sanity checks and avoid dropping system.query_log table [#50934](https://github.com/ClickHouse/ClickHouse/pull/50934) ([Azat Khuzhin](https://github.com/azat)). +* Fix tests for throttling by allowing more margin of error for trottling event [#50935](https://github.com/ClickHouse/ClickHouse/pull/50935) ([Azat Khuzhin](https://github.com/azat)). +* 01746_convert_type_with_default: Temporarily disable flaky test [#50937](https://github.com/ClickHouse/ClickHouse/pull/50937) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix the statless tests image for old commits [#50947](https://github.com/ClickHouse/ClickHouse/pull/50947) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix logic in `AsynchronousBoundedReadBuffer::seek` [#50952](https://github.com/ClickHouse/ClickHouse/pull/50952) ([Nikita Taranov](https://github.com/nickitat)). +* Uncomment flaky test (01746_convert_type_with_default) [#50954](https://github.com/ClickHouse/ClickHouse/pull/50954) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Fix keeper-client help message [#50965](https://github.com/ClickHouse/ClickHouse/pull/50965) ([pufit](https://github.com/pufit)). +* fix build issue on clang 15 [#50967](https://github.com/ClickHouse/ClickHouse/pull/50967) ([Chang chen](https://github.com/baibaichen)). +* Docs: Fix embedded video link [#50972](https://github.com/ClickHouse/ClickHouse/pull/50972) ([Robert Schulze](https://github.com/rschu1ze)). +* Change submodule capnproto to it's fork in ClickHouse [#50987](https://github.com/ClickHouse/ClickHouse/pull/50987) ([Kruglov Pavel](https://github.com/Avogar)). +* Attempt to make 01281_group_by_limit_memory_tracking not flaky [#50995](https://github.com/ClickHouse/ClickHouse/pull/50995) ([Dmitry Novik](https://github.com/novikd)). +* Fix flaky 02561_null_as_default_more_formats [#51001](https://github.com/ClickHouse/ClickHouse/pull/51001) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix flaky test_seekable_formats [#51002](https://github.com/ClickHouse/ClickHouse/pull/51002) ([Kruglov Pavel](https://github.com/Avogar)). +* Follow-up to [#50448](https://github.com/ClickHouse/ClickHouse/issues/50448) [#51006](https://github.com/ClickHouse/ClickHouse/pull/51006) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix a versions' tweak for tagged commits, improve version_helper [#51035](https://github.com/ClickHouse/ClickHouse/pull/51035) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Sqlancer has changed master to main [#51060](https://github.com/ClickHouse/ClickHouse/pull/51060) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Do not spam sqlancer build log [#51061](https://github.com/ClickHouse/ClickHouse/pull/51061) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Refactor IColumn::forEachSubcolumn to make it slightly harder to implement incorrectly [#51072](https://github.com/ClickHouse/ClickHouse/pull/51072) ([Michael Kolupaev](https://github.com/al13n321)). +* MaterializedMySQL: Rename materialize_with_ddl.py -> materialized_with_ddl [#51074](https://github.com/ClickHouse/ClickHouse/pull/51074) ([Val Doroshchuk](https://github.com/valbok)). +* Improve woboq browser report [#51077](https://github.com/ClickHouse/ClickHouse/pull/51077) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix for part_names_mutex used after destruction [#51099](https://github.com/ClickHouse/ClickHouse/pull/51099) ([Alexander Gololobov](https://github.com/davenger)). +* Fix ColumnConst::forEachSubcolumn missing from previous PR [#51102](https://github.com/ClickHouse/ClickHouse/pull/51102) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix the test 02783_parsedatetimebesteffort_syslog flakiness [#51112](https://github.com/ClickHouse/ClickHouse/pull/51112) ([Victor Krasnov](https://github.com/sirvickr)). +* Compatibility with clang-17 [#51114](https://github.com/ClickHouse/ClickHouse/pull/51114) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make more parallel get requests to ZooKeeper in system.zookeeper [#51118](https://github.com/ClickHouse/ClickHouse/pull/51118) ([Alexander Gololobov](https://github.com/davenger)). +* Fix 02703_max_local_write_bandwidth flakiness [#51120](https://github.com/ClickHouse/ClickHouse/pull/51120) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.5.3.24-stable [#51121](https://github.com/ClickHouse/ClickHouse/pull/51121) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.4.4.16-stable [#51122](https://github.com/ClickHouse/ClickHouse/pull/51122) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.3.4.17-lts [#51123](https://github.com/ClickHouse/ClickHouse/pull/51123) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.8.19.10-lts [#51124](https://github.com/ClickHouse/ClickHouse/pull/51124) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix typo [#51126](https://github.com/ClickHouse/ClickHouse/pull/51126) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Slightly better diagnostics [#51127](https://github.com/ClickHouse/ClickHouse/pull/51127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix in `MergeTreePrefetchedReadPool` [#51131](https://github.com/ClickHouse/ClickHouse/pull/51131) ([Nikita Taranov](https://github.com/nickitat)). +* Don't report table function accesses to system.errors [#51147](https://github.com/ClickHouse/ClickHouse/pull/51147) ([Raúl Marín](https://github.com/Algunenano)). +* Fix SQLancer branch name [#51148](https://github.com/ClickHouse/ClickHouse/pull/51148) ([Ilya Yatsishin](https://github.com/qoega)). +* Revert "Added ability to implicitly use file/hdfs/s3 table functions in clickhouse-local" [#51149](https://github.com/ClickHouse/ClickHouse/pull/51149) ([Alexander Tokmakov](https://github.com/tavplubix)). +* More profile events for fs cache [#51161](https://github.com/ClickHouse/ClickHouse/pull/51161) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Unforget to pass callback to readBigAt() in ParallelReadBuffer [#51165](https://github.com/ClickHouse/ClickHouse/pull/51165) ([Michael Kolupaev](https://github.com/al13n321)). +* Update README.md [#51179](https://github.com/ClickHouse/ClickHouse/pull/51179) ([Tyler Hannan](https://github.com/tylerhannan)). +* Update exception message [#51187](https://github.com/ClickHouse/ClickHouse/pull/51187) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Split long test 02149_schema_inference_formats_with_schema into several tests to avoid timeout in debug [#51197](https://github.com/ClickHouse/ClickHouse/pull/51197) ([Kruglov Pavel](https://github.com/Avogar)). +* Avoid initializing DateLUT from emptyArray function registration [#51199](https://github.com/ClickHouse/ClickHouse/pull/51199) ([Alexander Gololobov](https://github.com/davenger)). +* Suppress check for covered parts in ZooKeeper [#51207](https://github.com/ClickHouse/ClickHouse/pull/51207) ([Alexander Tokmakov](https://github.com/tavplubix)). +* One more profile event for fs cache [#51223](https://github.com/ClickHouse/ClickHouse/pull/51223) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Typo: passowrd_sha256_hex --> password_sha256_hex [#51233](https://github.com/ClickHouse/ClickHouse/pull/51233) ([Robert Schulze](https://github.com/rschu1ze)). +* Introduce settings enum field with auto-generated values list [#51237](https://github.com/ClickHouse/ClickHouse/pull/51237) ([Sergei Trifonov](https://github.com/serxa)). +* Drop session if we fail to get Keeper API version [#51238](https://github.com/ClickHouse/ClickHouse/pull/51238) ([Alexander Gololobov](https://github.com/davenger)). +* Revert "Fix a crash in s3 and s3Cluster functions" [#51239](https://github.com/ClickHouse/ClickHouse/pull/51239) ([Alexander Tokmakov](https://github.com/tavplubix)). +* fix flaky `AsyncLoader` destructor [#51245](https://github.com/ClickHouse/ClickHouse/pull/51245) ([Sergei Trifonov](https://github.com/serxa)). +* Docs: little cleanup of configuration-files.md [#51249](https://github.com/ClickHouse/ClickHouse/pull/51249) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix a stupid bug on Replicated database recovery [#51252](https://github.com/ClickHouse/ClickHouse/pull/51252) ([Alexander Tokmakov](https://github.com/tavplubix)). +* FileCache: tryReserve() slight improvement [#51259](https://github.com/ClickHouse/ClickHouse/pull/51259) ([Igor Nikonov](https://github.com/devcrafter)). +* Ugly hotfix for "terminate on uncaught exception" in WriteBufferFromOStream [#51265](https://github.com/ClickHouse/ClickHouse/pull/51265) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Avoid too many calls to Poco::Logger::get [#51266](https://github.com/ClickHouse/ClickHouse/pull/51266) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update version_date.tsv and changelogs after v23.3.5.9-lts [#51269](https://github.com/ClickHouse/ClickHouse/pull/51269) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Better reporting of broken parts [#51270](https://github.com/ClickHouse/ClickHouse/pull/51270) ([Anton Popov](https://github.com/CurtizJ)). +* Update ext-dict-functions.md [#51283](https://github.com/ClickHouse/ClickHouse/pull/51283) ([Mike Kot](https://github.com/myrrc)). +* Disable table structure check for secondary queries from Replicated db [#51284](https://github.com/ClickHouse/ClickHouse/pull/51284) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Define Thrift version for parquet and use correct arrow version [#51285](https://github.com/ClickHouse/ClickHouse/pull/51285) ([Kruglov Pavel](https://github.com/Avogar)). +* Restore Azure build on ARM [#51288](https://github.com/ClickHouse/ClickHouse/pull/51288) ([Robert Schulze](https://github.com/rschu1ze)). +* Query Cache: Un-comment settings in server cfg [#51294](https://github.com/ClickHouse/ClickHouse/pull/51294) ([Robert Schulze](https://github.com/rschu1ze)). +* Require more checks [#51295](https://github.com/ClickHouse/ClickHouse/pull/51295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix metadata loading test [#51297](https://github.com/ClickHouse/ClickHouse/pull/51297) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Scratch the strange Python code [#51302](https://github.com/ClickHouse/ClickHouse/pull/51302) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#47865](https://github.com/ClickHouse/ClickHouse/issues/47865) [#51306](https://github.com/ClickHouse/ClickHouse/pull/51306) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#48894](https://github.com/ClickHouse/ClickHouse/issues/48894) [#51307](https://github.com/ClickHouse/ClickHouse/pull/51307) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#48676](https://github.com/ClickHouse/ClickHouse/issues/48676) [#51308](https://github.com/ClickHouse/ClickHouse/pull/51308) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix long test `functions_bad_arguments` [#51310](https://github.com/ClickHouse/ClickHouse/pull/51310) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Unify merge predicate [#51344](https://github.com/ClickHouse/ClickHouse/pull/51344) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix using locks in ProcessList [#51348](https://github.com/ClickHouse/ClickHouse/pull/51348) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add a test for [#42631](https://github.com/ClickHouse/ClickHouse/issues/42631) [#51353](https://github.com/ClickHouse/ClickHouse/pull/51353) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix performance tests due to warnings from jemalloc about Per-CPU arena disabled [#51362](https://github.com/ClickHouse/ClickHouse/pull/51362) ([Azat Khuzhin](https://github.com/azat)). +* Fix "merge_truncate_long" test [#51369](https://github.com/ClickHouse/ClickHouse/pull/51369) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Increase timeout of Fast Test [#51372](https://github.com/ClickHouse/ClickHouse/pull/51372) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad tests for DNS [#51374](https://github.com/ClickHouse/ClickHouse/pull/51374) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Attempt to fix the `relax_too_many_parts` test [#51375](https://github.com/ClickHouse/ClickHouse/pull/51375) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix MySQL test in Debug mode [#51376](https://github.com/ClickHouse/ClickHouse/pull/51376) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `01018_Distributed__shard_num` [#51377](https://github.com/ClickHouse/ClickHouse/pull/51377) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix "logical error" in addressToLineWithInlines [#51379](https://github.com/ClickHouse/ClickHouse/pull/51379) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 01280_ttl_where_group_by [#51380](https://github.com/ClickHouse/ClickHouse/pull/51380) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Attempt to fix `test_ssl_cert_authentication` [#51384](https://github.com/ClickHouse/ClickHouse/pull/51384) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert "Merge pull request [#50951](https://github.com/ClickHouse/ClickHouse/issues/50951) from ZhiguoZh/20230607-toyear-fix" [#51390](https://github.com/ClickHouse/ClickHouse/pull/51390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Two tests are twice longer in average with Analyzer and sometimes failing [#51391](https://github.com/ClickHouse/ClickHouse/pull/51391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix 00899_long_attach_memory_limit [#51395](https://github.com/ClickHouse/ClickHouse/pull/51395) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 01293_optimize_final_force [#51396](https://github.com/ClickHouse/ClickHouse/pull/51396) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 02481_parquet_list_monotonically_increasing_offsets [#51397](https://github.com/ClickHouse/ClickHouse/pull/51397) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 02497_trace_events_stress_long [#51398](https://github.com/ClickHouse/ClickHouse/pull/51398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix broken labeling for `manual approve` [#51405](https://github.com/ClickHouse/ClickHouse/pull/51405) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix parts lifetime in `MergeTreeTransaction` [#51407](https://github.com/ClickHouse/ClickHouse/pull/51407) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix flaky test test_skip_empty_files [#51409](https://github.com/ClickHouse/ClickHouse/pull/51409) ([Kruglov Pavel](https://github.com/Avogar)). +* fix flacky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). +* Update README.md [#51413](https://github.com/ClickHouse/ClickHouse/pull/51413) ([Tyler Hannan](https://github.com/tylerhannan)). +* Replace try/catch logic in hasTokenOrNull() by something more lightweight [#51425](https://github.com/ClickHouse/ClickHouse/pull/51425) ([Robert Schulze](https://github.com/rschu1ze)). +* Add retries to `tlsv1_3` tests [#51434](https://github.com/ClickHouse/ClickHouse/pull/51434) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Update exception message [#51440](https://github.com/ClickHouse/ClickHouse/pull/51440) ([Kseniia Sumarokova](https://github.com/kssenii)). +* fs cache: add check for intersecting ranges [#51444](https://github.com/ClickHouse/ClickHouse/pull/51444) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Slightly better code around packets for parallel replicas [#51451](https://github.com/ClickHouse/ClickHouse/pull/51451) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Update system_warnings test [#51453](https://github.com/ClickHouse/ClickHouse/pull/51453) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Many fixes [#51455](https://github.com/ClickHouse/ClickHouse/pull/51455) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 01605_adaptive_granularity_block_borders [#51457](https://github.com/ClickHouse/ClickHouse/pull/51457) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try fix flaky 02497_storage_file_reader_selection [#51468](https://github.com/ClickHouse/ClickHouse/pull/51468) ([Kruglov Pavel](https://github.com/Avogar)). +* Try making Keeper in `DatabaseReplicated` tests more stable [#51473](https://github.com/ClickHouse/ClickHouse/pull/51473) ([Antonio Andelic](https://github.com/antonio2368)). +* Convert 02003_memory_limit_in_client from expect to sh test (to fix flakiness) [#51475](https://github.com/ClickHouse/ClickHouse/pull/51475) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_disk_over_web_server [#51476](https://github.com/ClickHouse/ClickHouse/pull/51476) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Delay shutdown of system and temporary databases [#51479](https://github.com/ClickHouse/ClickHouse/pull/51479) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix memory leakage in CompressionCodecDeflateQpl [#51480](https://github.com/ClickHouse/ClickHouse/pull/51480) ([Vitaly Baranov](https://github.com/vitlibar)). +* Increase retries in test_multiple_disks/test.py::test_start_stop_moves [#51482](https://github.com/ClickHouse/ClickHouse/pull/51482) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix race in BoundedReadBuffer [#51484](https://github.com/ClickHouse/ClickHouse/pull/51484) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky unit test [#51485](https://github.com/ClickHouse/ClickHouse/pull/51485) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test `test_host_regexp_multiple_ptr_records` [#51506](https://github.com/ClickHouse/ClickHouse/pull/51506) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add a comment [#51517](https://github.com/ClickHouse/ClickHouse/pull/51517) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make `test_ssl_cert_authentication` similar to `test_tlvs1_3` [#51520](https://github.com/ClickHouse/ClickHouse/pull/51520) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix duplicate storage set logical error. [#51521](https://github.com/ClickHouse/ClickHouse/pull/51521) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update test_storage_postgresql/test.py::test_concurrent_queries [#51523](https://github.com/ClickHouse/ClickHouse/pull/51523) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix FATAL: query context is not detached from thread group [#51540](https://github.com/ClickHouse/ClickHouse/pull/51540) ([Igor Nikonov](https://github.com/devcrafter)). +* Update version_date.tsv and changelogs after v23.3.6.7-lts [#51548](https://github.com/ClickHouse/ClickHouse/pull/51548) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Decoupled commits from [#51180](https://github.com/ClickHouse/ClickHouse/issues/51180) for backports [#51561](https://github.com/ClickHouse/ClickHouse/pull/51561) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Try to fix deadlock in ZooKeeper client [#51563](https://github.com/ClickHouse/ClickHouse/pull/51563) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Retry chroot creation in ZK before stateless tests [#51585](https://github.com/ClickHouse/ClickHouse/pull/51585) ([Antonio Andelic](https://github.com/antonio2368)). +* use timeout instead trap in 01443_merge_truncate_long.sh [#51593](https://github.com/ClickHouse/ClickHouse/pull/51593) ([Sema Checherinda](https://github.com/CheSema)). +* Update version_date.tsv and changelogs after v23.5.4.25-stable [#51604](https://github.com/ClickHouse/ClickHouse/pull/51604) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix MergeTreeMarksLoader segfaulting if marks file is longer than expected [#51636](https://github.com/ClickHouse/ClickHouse/pull/51636) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelogs after v23.4.5.22-stable [#51638](https://github.com/ClickHouse/ClickHouse/pull/51638) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.3.7.5-lts [#51639](https://github.com/ClickHouse/ClickHouse/pull/51639) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update parts.md [#51643](https://github.com/ClickHouse/ClickHouse/pull/51643) ([Ramazan Polat](https://github.com/ramazanpolat)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4ca5d1d7497..2a098d8c1da 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.6.1.1524-stable 2023-06-30 v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10