From 97f4ec2adbf614842e9b16badb69a8d5c642abe0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 17 Jul 2024 16:59:35 +0200 Subject: [PATCH 01/40] Read cgroup memory usage in async metrics thread --- programs/keeper/Keeper.cpp | 26 ++-- programs/server/Server.cpp | 34 ++--- src/Common/AsynchronousMetrics.cpp | 13 +- src/Common/AsynchronousMetrics.h | 5 +- src/Common/CgroupsMemoryUsageObserver.cpp | 127 ++++-------------- src/Common/CgroupsMemoryUsageObserver.h | 19 +-- src/Common/MemoryTracker.cpp | 38 ++---- src/Common/MemoryTracker.h | 4 +- .../KeeperAsynchronousMetrics.cpp | 2 +- src/Interpreters/Context.cpp | 13 ++ src/Interpreters/Context.h | 4 + .../ServerAsynchronousMetrics.cpp | 2 +- 12 files changed, 112 insertions(+), 175 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 44c2daa33ad..3f6020ad48c 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -399,6 +399,18 @@ try registerDisks(/*global_skip_access_check=*/false); + auto cgroups_memory_observer_wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); + try + { + auto cgroups_reader = createCgroupsReader(); + global_context->setCgroupsReader(createCgroupsReader()); + } + catch (...) + { + if (cgroups_memory_observer_wait_time != 0) + tryLogCurrentException(log, "Failed to create cgroups reader"); + } + /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( global_context, @@ -622,21 +634,19 @@ try main_config_reloader->start(); std::optional cgroups_memory_usage_observer; - try + if (cgroups_memory_observer_wait_time != 0) { - auto wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); - if (wait_time != 0) + auto cgroups_reader = global_context->getCgroupsReader(); + if (cgroups_reader) { - cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); + cgroups_memory_usage_observer.emplace(std::chrono::seconds(cgroups_memory_observer_wait_time), global_context->getCgroupsReader()); /// Not calling cgroups_memory_usage_observer->setLimits() here (as for the normal ClickHouse server) because Keeper controls /// its memory usage by other means (via setting 'max_memory_usage_soft_limit'). cgroups_memory_usage_observer->setOnMemoryAmountAvailableChangedFn([&]() { main_config_reloader->reload(); }); cgroups_memory_usage_observer->startThread(); } - } - catch (Exception &) - { - tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); + else + LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 053ddaf8d8b..c52b1e037ec 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -897,6 +897,17 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + try + { + auto cgroups_reader = createCgroupsReader(); + global_context->setCgroupsReader(createCgroupsReader()); + } + catch (...) + { + if (server_settings.cgroups_memory_usage_observer_wait_time != 0) + tryLogCurrentException(log, "Failed to create cgroups reader"); + } + /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -1456,15 +1467,13 @@ try } std::optional cgroups_memory_usage_observer; - try + if (auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; wait_time != 0) { - auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; - if (wait_time != 0) - cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); - } - catch (Exception &) - { - tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); + auto cgroups_reader = global_context->getCgroupsReader(); + if (cgroups_reader) + cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time), std::move(cgroups_reader)); + else + LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); } std::string cert_path = config().getString("openSSL.server.certificateFile", ""); @@ -1532,15 +1541,6 @@ try total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - if (cgroups_memory_usage_observer) - { - double hard_limit_ratio = new_server_settings.cgroup_memory_watcher_hard_limit_ratio; - double soft_limit_ratio = new_server_settings.cgroup_memory_watcher_soft_limit_ratio; - cgroups_memory_usage_observer->setMemoryUsageLimits( - static_cast(max_server_memory_usage * hard_limit_ratio), - static_cast(max_server_memory_usage * soft_limit_ratio)); - } - size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit; size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio); diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 6309f6079f6..0953ad88697 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -57,10 +57,12 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + std::shared_ptr cgroups_reader_) : update_period(update_period_seconds) , log(getLogger("AsynchronousMetrics")) , protocol_server_metrics_func(protocol_server_metrics_func_) + , cgroups_reader(std::move(cgroups_reader_)) { #if defined(OS_LINUX) openFileIfExists("/proc/meminfo", meminfo); @@ -669,6 +671,13 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) free_memory_in_allocator_arenas = je_malloc_pdirty * getPageSize(); #endif + if (cgroups_reader != nullptr) + { + rss = cgroups_reader->readMemoryUsage(); + new_values["CgroupsMemoryUsage"] = { rss, + "The amount of physical memory used by the server process, reported by cgroups." }; + } + Int64 difference = rss - amount; /// Log only if difference is high. This is for convenience. The threshold is arbitrary. @@ -681,7 +690,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) ReadableSize(rss), ReadableSize(difference)); - MemoryTracker::setRSS(rss, free_memory_in_allocator_arenas); + MemoryTracker::setRSS(rss, /*has_free_memory_in_allocator_arenas_=*/free_memory_in_allocator_arenas > 0); } } diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 10a972d2458..0b110f41fc3 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -68,7 +69,8 @@ public: AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + std::shared_ptr cgroups_reader_); virtual ~AsynchronousMetrics(); @@ -91,6 +93,7 @@ private: virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; + std::shared_ptr cgroups_reader; std::unique_ptr thread; diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 02bde0d80b7..b12845df098 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -17,13 +17,6 @@ #include #include -#include "config.h" -#if USE_JEMALLOC -# include -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) -#endif - using namespace DB; namespace fs = std::filesystem; @@ -155,15 +148,21 @@ std::optional getCgroupsV1Path() return {default_cgroups_mount / "memory"}; } -std::pair getCgroupsPath() +enum class CgroupsVersion : uint8_t +{ + V1, + V2 +}; + +std::pair getCgroupsPath() { auto v2_path = getCgroupsV2Path(); if (v2_path.has_value()) - return {*v2_path, CgroupsMemoryUsageObserver::CgroupsVersion::V2}; + return {*v2_path, CgroupsVersion::V2}; auto v1_path = getCgroupsV1Path(); if (v1_path.has_value()) - return {*v1_path, CgroupsMemoryUsageObserver::CgroupsVersion::V1}; + return {*v1_path, CgroupsVersion::V1}; throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); } @@ -173,22 +172,29 @@ std::pair getCgroupsPat namespace DB { -CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) - : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) +std::shared_ptr createCgroupsReader() { const auto [cgroup_path, version] = getCgroupsPath(); + LOG_INFO( + getLogger("CgroupsReader"), + "Will create cgroup reader from '{}' (cgroups version: {})", + cgroup_path, + (version == CgroupsVersion::V1) ? "v1" : "v2"); if (version == CgroupsVersion::V2) - cgroup_reader = std::make_unique(cgroup_path); + return std::make_shared(cgroup_path); else - cgroup_reader = std::make_unique(cgroup_path); + { + chassert(version == CgroupsVersion::V1); + return std::make_shared(cgroup_path); + } - LOG_INFO( - log, - "Will read the current memory usage from '{}' (cgroups version: {}), wait time is {} sec", - cgroup_path, - (version == CgroupsVersion::V1) ? "v1" : "v2", - wait_time.count()); +} + +CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_) + : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(std::move(cgroups_reader_)) +{ + cgroups_reader = createCgroupsReader(); } CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() @@ -196,58 +202,6 @@ CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() stopThread(); } -void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_) -{ - std::lock_guard limit_lock(limit_mutex); - - if (hard_limit_ == hard_limit && soft_limit_ == soft_limit) - return; - - hard_limit = hard_limit_; - soft_limit = soft_limit_; - - on_hard_limit = [this, hard_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded hard memory limit ({})", ReadableSize(hard_limit_)); - - /// Update current usage in memory tracker. Also reset free_memory_in_allocator_arenas to zero though we don't know if they are - /// really zero. Trying to avoid OOM ... - MemoryTracker::setRSS(hard_limit_, 0); - } - else - { - LOG_INFO(log, "Dropped below hard memory limit ({})", ReadableSize(hard_limit_)); - } - }; - - on_soft_limit = [this, soft_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded soft memory limit ({})", ReadableSize(soft_limit_)); - -# if USE_JEMALLOC - LOG_INFO(log, "Purging jemalloc arenas"); - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); -# endif - /// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them. - uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); - MemoryTracker::setRSS(memory_usage, 0); - - LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage)); - } - else - { - LOG_INFO(log, "Dropped below soft memory limit ({})", ReadableSize(soft_limit_)); - } - }; - - LOG_INFO(log, "Set new limits, soft limit: {}, hard limit: {}", ReadableSize(soft_limit_), ReadableSize(hard_limit_)); -} - void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_) { std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); @@ -301,35 +255,6 @@ void CgroupsMemoryUsageObserver::runThread() std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); on_memory_amount_available_changed(); } - - std::lock_guard limit_lock(limit_mutex); - if (soft_limit > 0 && hard_limit > 0) - { - uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); - if (memory_usage > hard_limit) - { - if (last_memory_usage <= hard_limit) - on_hard_limit(true); - } - else - { - if (last_memory_usage > hard_limit) - on_hard_limit(false); - } - - if (memory_usage > soft_limit) - { - if (last_memory_usage <= soft_limit) - on_soft_limit(true); - } - else - { - if (last_memory_usage > soft_limit) - on_soft_limit(false); - } - last_memory_usage = memory_usage; - } } catch (...) { diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index b848a2bff3c..078307a6fa0 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -16,6 +16,8 @@ struct ICgroupsReader virtual uint64_t readMemoryUsage() = 0; }; +std::shared_ptr createCgroupsReader(); + /// Does two things: /// 1. Periodically reads the memory usage of the process from Linux cgroups. /// You can specify soft or hard memory limits: @@ -35,19 +37,11 @@ struct ICgroupsReader class CgroupsMemoryUsageObserver { public: - using OnMemoryLimitFn = std::function; using OnMemoryAmountAvailableChangedFn = std::function; - enum class CgroupsVersion : uint8_t - { - V1, - V2 - }; - - explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_); + explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_); ~CgroupsMemoryUsageObserver(); - void setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_); void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_); void startThread(); @@ -58,22 +52,17 @@ private: const std::chrono::seconds wait_time; std::mutex limit_mutex; - size_t hard_limit TSA_GUARDED_BY(limit_mutex) = 0; - size_t soft_limit TSA_GUARDED_BY(limit_mutex) = 0; - OnMemoryLimitFn on_hard_limit TSA_GUARDED_BY(limit_mutex); - OnMemoryLimitFn on_soft_limit TSA_GUARDED_BY(limit_mutex); std::mutex memory_amount_available_changed_mutex; OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed TSA_GUARDED_BY(memory_amount_available_changed_mutex); - uint64_t last_memory_usage = 0; /// how much memory does the process use uint64_t last_available_memory_amount; /// how much memory can the process use void stopThread(); void runThread(); - std::unique_ptr cgroup_reader; + std::shared_ptr cgroups_reader; std::mutex thread_mutex; std::condition_variable cond; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 28cfa98666a..eaf34d87ec5 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -26,7 +26,6 @@ #endif #include -#include #include #include #include @@ -123,7 +122,7 @@ static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; MemoryTracker total_memory_tracker(nullptr, VariableContext::Global); MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false); -std::atomic MemoryTracker::free_memory_in_allocator_arenas; +std::atomic MemoryTracker::has_free_memory_in_allocator_arenas; MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} @@ -204,7 +203,7 @@ void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused] LOG_TEST(getLogger("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); #else - return; /// Avoid trash logging in release builds + /// Avoid trash logging in release builds #endif } @@ -294,33 +293,18 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } - Int64 limit_to_check = current_hard_limit; - #if USE_JEMALLOC - if (level == VariableContext::Global && allow_use_jemalloc_memory.load(std::memory_order_relaxed)) + if (level == VariableContext::Global && will_be > soft_limit.load(std::memory_order_relaxed) + && has_free_memory_in_allocator_arenas.exchange(false)) { - /// Jemalloc arenas may keep some extra memory. - /// This memory was substucted from RSS to decrease memory drift. - /// In case memory is close to limit, try to pugre the arenas. - /// This is needed to avoid OOM, because some allocations are directly done with mmap. - Int64 current_free_memory_in_allocator_arenas = free_memory_in_allocator_arenas.load(std::memory_order_relaxed); - - if (current_free_memory_in_allocator_arenas > 0 && current_hard_limit && current_free_memory_in_allocator_arenas + will_be > current_hard_limit) - { - if (free_memory_in_allocator_arenas.exchange(-current_free_memory_in_allocator_arenas) > 0) - { - Stopwatch watch; - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); - } - } - - limit_to_check += abs(current_free_memory_in_allocator_arenas); + Stopwatch watch; + mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); } #endif - if (unlikely(current_hard_limit && will_be > limit_to_check)) + if (unlikely(current_hard_limit && will_be > current_hard_limit)) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { @@ -526,11 +510,11 @@ void MemoryTracker::reset() } -void MemoryTracker::setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_) +void MemoryTracker::setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_) { Int64 new_amount = rss_; total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); - free_memory_in_allocator_arenas.store(free_memory_in_allocator_arenas_, std::memory_order_relaxed); + has_free_memory_in_allocator_arenas.store(has_free_memory_in_allocator_arenas_, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index fd32b631774..48d02fd1fc6 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -59,7 +59,7 @@ private: std::atomic profiler_limit {0}; std::atomic_bool allow_use_jemalloc_memory {true}; - static std::atomic free_memory_in_allocator_arenas; + static std::atomic has_free_memory_in_allocator_arenas; Int64 profiler_step = 0; @@ -252,7 +252,7 @@ public: /// Reset current counter to an RSS value. /// Jemalloc may have pre-allocated arenas, they are accounted in RSS. /// We can free this arenas in case of exception to avoid OOM. - static void setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_); + static void setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 86166ffe31b..3e404b7152b 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -115,7 +115,7 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, context_->getCgroupsReader()), context(std::move(context_)) { } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 2602afd8b78..771b8e9e558 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -405,6 +406,8 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr cluster_discovery TSA_GUARDED_BY(clusters_mutex); size_t clusters_version TSA_GUARDED_BY(clusters_mutex) = 0; + std::shared_ptr cgroups_reader; + /// No lock required for async_insert_queue modified only during initialization std::shared_ptr async_insert_queue; @@ -5635,6 +5638,16 @@ const ServerSettings & Context::getServerSettings() const return shared->server_settings; } +void Context::setCgroupsReader(std::shared_ptr cgroups_reader_) +{ + shared->cgroups_reader = std::move(cgroups_reader_); +} + +std::shared_ptr Context::getCgroupsReader() const +{ + return shared->cgroups_reader; +} + uint64_t HTTPContext::getMaxHstsAge() const { return context->getSettingsRef().hsts_max_age; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 284cac50769..f183a72e8e2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -150,6 +150,7 @@ class ServerType; template class MergeTreeBackgroundExecutor; class AsyncLoader; +struct ICgroupsReader; struct TemporaryTableHolder; using TemporaryTablesMapping = std::map>; @@ -1344,6 +1345,9 @@ public: const ServerSettings & getServerSettings() const; + void setCgroupsReader(std::shared_ptr cgroups_reader_); + std::shared_ptr getCgroupsReader() const; + private: std::shared_ptr getSettingsConstraintsAndCurrentProfilesWithLock() const; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 872a9f864df..6ee0168bede 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -57,7 +57,7 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( unsigned heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : WithContext(global_context_) - , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, getContext()->getCgroupsReader()) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check From 21009577d867d493a6ceda48987c060498774f94 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 09:01:08 +0200 Subject: [PATCH 02/40] Dedicated memory background thread --- programs/server/Server.cpp | 39 ++++++++++++++++++++++++--- src/CMakeLists.txt | 2 +- src/Common/AsynchronousMetrics.cpp | 43 ------------------------------ src/Common/Jemalloc.cpp | 16 ----------- src/Common/Jemalloc.h | 29 ++++++++++++++++++++ src/Common/MemoryTracker.cpp | 33 +++++++++-------------- src/Common/MemoryTracker.h | 1 - 7 files changed, 78 insertions(+), 85 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c52b1e037ec..ca46338d1c1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -447,9 +447,12 @@ void checkForUsersNotInMainConfig( } } +namespace +{ + /// Unused in other builds #if defined(OS_LINUX) -static String readLine(const String & path) +String readLine(const String & path) { ReadBufferFromFile in(path); String contents; @@ -457,7 +460,7 @@ static String readLine(const String & path) return contents; } -static int readNumber(const String & path) +int readNumber(const String & path) { ReadBufferFromFile in(path); int result; @@ -467,7 +470,7 @@ static int readNumber(const String & path) #endif -static void sanityChecks(Server & server) +void sanityChecks(Server & server) { std::string data_path = getCanonicalPath(server.config().getString("path", DBMS_DEFAULT_PATH)); std::string logs_path = server.config().getString("logger.log", ""); @@ -588,6 +591,31 @@ static void sanityChecks(Server & server) } } +[[noreturn]] void backgroundMemoryThread() +{ + std::mutex mutex; + std::condition_variable cv; + + std::unique_lock lock(mutex); + while (true) + { + cv.wait_for(lock, std::chrono::microseconds(200)); + uint64_t epoch = 0; + mallctl("epoch", nullptr, nullptr, &epoch, sizeof(epoch)); + auto maybe_resident = getJemallocValue("stats.resident"); + if (!maybe_resident.has_value()) + continue; + + Int64 resident = *maybe_resident; + //LOG_INFO(getLogger("JEmalloc"), "Resident {}", ReadableSize(resident)); + MemoryTracker::setRSS(resident, false); + if (resident > total_memory_tracker.getHardLimit()) + purgeJemallocArenas(); + } +} + +} + void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log) { try @@ -877,6 +905,11 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } + ThreadFromGlobalPool background_memory_thread([] + { + backgroundMemoryThread(); + }); + Poco::ThreadPool server_pool( /* minCapacity */3, /* maxCapacity */server_settings.max_connections, diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d985595154c..bfa41eacea1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -174,7 +174,7 @@ add_library (clickhouse_new_delete STATIC Common/new_delete.cpp) target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io) if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::jemalloc) - target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::jemalloc) target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc) endif() diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 0953ad88697..b3e53c29d4a 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -649,49 +649,6 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) "The amount of virtual memory mapped for the use of stack and for the allocated memory, in bytes." " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; - - /// We must update the value of total_memory_tracker periodically. - /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. - /// See https://github.com/ClickHouse/ClickHouse/issues/10293 - { - Int64 amount = total_memory_tracker.get(); - Int64 peak = total_memory_tracker.getPeak(); - Int64 rss = data.resident; - Int64 free_memory_in_allocator_arenas = 0; - -#if USE_JEMALLOC - /// According to jemalloc man, pdirty is: - /// - /// Number of pages within unused extents that are potentially - /// dirty, and for which madvise() or similar has not been called. - /// - /// So they will be subtracted from RSS to make accounting more - /// accurate, since those pages are not really RSS but a memory - /// that can be used at anytime via jemalloc. - free_memory_in_allocator_arenas = je_malloc_pdirty * getPageSize(); -#endif - - if (cgroups_reader != nullptr) - { - rss = cgroups_reader->readMemoryUsage(); - new_values["CgroupsMemoryUsage"] = { rss, - "The amount of physical memory used by the server process, reported by cgroups." }; - } - - Int64 difference = rss - amount; - - /// Log only if difference is high. This is for convenience. The threshold is arbitrary. - if (difference >= 1048576 || difference <= -1048576) - LOG_TRACE(log, - "MemoryTracking: was {}, peak {}, free memory in arenas {}, will set to {} (RSS), difference: {}", - ReadableSize(amount), - ReadableSize(peak), - ReadableSize(free_memory_in_allocator_arenas), - ReadableSize(rss), - ReadableSize(difference)); - - MemoryTracker::setRSS(rss, /*has_free_memory_in_allocator_arenas_=*/free_memory_in_allocator_arenas > 0); - } } { diff --git a/src/Common/Jemalloc.cpp b/src/Common/Jemalloc.cpp index d7cc246db6a..d8ff9268cca 100644 --- a/src/Common/Jemalloc.cpp +++ b/src/Common/Jemalloc.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #define STRINGIFY_HELPER(x) #x #define STRINGIFY(x) STRINGIFY_HELPER(x) @@ -26,7 +25,6 @@ namespace ErrorCodes void purgeJemallocArenas() { - LOG_TRACE(getLogger("SystemJemalloc"), "Purging unused memory"); Stopwatch watch; mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); @@ -46,20 +44,6 @@ void checkJemallocProfilingEnabled() "set: MALLOC_CONF=background_thread:true,prof:true"); } -template -void setJemallocValue(const char * name, T value) -{ - T old_value; - size_t old_value_size = sizeof(T); - if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) - { - LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); - return; - } - - LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); -} - void setJemallocProfileActive(bool value) { checkJemallocProfilingEnabled(); diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 499a906fd3d..0c533711f78 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -5,6 +5,8 @@ #if USE_JEMALLOC #include +#include +#include namespace DB { @@ -21,6 +23,33 @@ void setJemallocBackgroundThreads(bool enabled); void setJemallocMaxBackgroundThreads(size_t max_threads); +template +void setJemallocValue(const char * name, T value) +{ + T old_value; + size_t old_value_size = sizeof(T); + if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) + { + LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); + return; + } + + LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); +} + +template +std::optional getJemallocValue(const char * name) +{ + T value; + size_t value_size = sizeof(T); + if (mallctl(name, &value, &value_size, nullptr, 0)) + { + LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); + return std::nullopt; + } + return value; +} + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index eaf34d87ec5..a541eeeb25f 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -20,9 +20,6 @@ #if USE_JEMALLOC # include -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) - #endif #include @@ -126,11 +123,11 @@ std::atomic MemoryTracker::has_free_memory_in_allocator_arenas; MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} + MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_, bool log_peak_memory_usage_in_destructor_) - : parent(parent_) - , log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_) - , level(level_) -{} + : parent(parent_), log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_), level(level_) +{ +} MemoryTracker::~MemoryTracker() { @@ -200,8 +197,12 @@ void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused] return; MemoryTrackerBlockerInThread blocker(VariableContext::Global); - LOG_TEST(getLogger("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " - "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); + LOG_TEST( + getLogger("MemoryTracker"), + "Too big allocation ({} bytes) without checking memory limits, " + "it may lead to OOM. Stack trace: {}", + size, + StackTrace().toString()); #else /// Avoid trash logging in release builds #endif @@ -293,17 +294,6 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } -#if USE_JEMALLOC - if (level == VariableContext::Global && will_be > soft_limit.load(std::memory_order_relaxed) - && has_free_memory_in_allocator_arenas.exchange(false)) - { - Stopwatch watch; - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); - } -#endif - if (unlikely(current_hard_limit && will_be > current_hard_limit)) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) @@ -513,7 +503,8 @@ void MemoryTracker::reset() void MemoryTracker::setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_) { Int64 new_amount = rss_; - total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); + if (rss_) + total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); has_free_memory_in_allocator_arenas.store(has_free_memory_in_allocator_arenas_, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 48d02fd1fc6..257ed7d0629 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include From 9a43183eb39dc056186432e6478e050db5045ecf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 10:31:24 +0200 Subject: [PATCH 03/40] Finish background memory thread --- programs/keeper/Keeper.cpp | 30 ++++------ programs/server/Server.cpp | 59 ++++--------------- src/Common/AsynchronousMetrics.cpp | 19 ++---- src/Common/AsynchronousMetrics.h | 3 +- src/Common/CgroupsMemoryUsageObserver.cpp | 8 +-- src/Common/CgroupsMemoryUsageObserver.h | 24 +++----- src/Common/Jemalloc.h | 42 +++++++++---- src/Common/MemoryTracker.cpp | 10 +--- src/Common/MemoryTracker.h | 13 +--- src/Common/MemoryWorker.cpp | 49 +++++++++++++++ src/Common/MemoryWorker.h | 34 +++++++++++ .../KeeperAsynchronousMetrics.cpp | 2 +- src/Core/ServerSettings.h | 1 + src/Interpreters/Context.cpp | 12 ---- src/Interpreters/Context.h | 3 - .../ServerAsynchronousMetrics.cpp | 2 +- .../System/StorageSystemServerSettings.cpp | 1 - tests/integration/test_memory_limit/test.py | 1 - 18 files changed, 158 insertions(+), 155 deletions(-) create mode 100644 src/Common/MemoryWorker.cpp create mode 100644 src/Common/MemoryWorker.h diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 3f6020ad48c..87f126a0046 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -371,6 +372,8 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 100)); + static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); @@ -399,18 +402,6 @@ try registerDisks(/*global_skip_access_check=*/false); - auto cgroups_memory_observer_wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); - try - { - auto cgroups_reader = createCgroupsReader(); - global_context->setCgroupsReader(createCgroupsReader()); - } - catch (...) - { - if (cgroups_memory_observer_wait_time != 0) - tryLogCurrentException(log, "Failed to create cgroups reader"); - } - /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( global_context, @@ -634,21 +625,22 @@ try main_config_reloader->start(); std::optional cgroups_memory_usage_observer; - if (cgroups_memory_observer_wait_time != 0) + try { - auto cgroups_reader = global_context->getCgroupsReader(); - if (cgroups_reader) + auto wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); + if (wait_time != 0) { - cgroups_memory_usage_observer.emplace(std::chrono::seconds(cgroups_memory_observer_wait_time), global_context->getCgroupsReader()); + cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); /// Not calling cgroups_memory_usage_observer->setLimits() here (as for the normal ClickHouse server) because Keeper controls /// its memory usage by other means (via setting 'max_memory_usage_soft_limit'). cgroups_memory_usage_observer->setOnMemoryAmountAvailableChangedFn([&]() { main_config_reloader->reload(); }); cgroups_memory_usage_observer->startThread(); } - else - LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); } - + catch (Exception &) + { + tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); + } LOG_INFO(log, "Ready for connections."); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ca46338d1c1..ae445477c3a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -24,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -110,6 +110,8 @@ #include #include +#include + #include "config.h" #include @@ -591,29 +593,6 @@ void sanityChecks(Server & server) } } -[[noreturn]] void backgroundMemoryThread() -{ - std::mutex mutex; - std::condition_variable cv; - - std::unique_lock lock(mutex); - while (true) - { - cv.wait_for(lock, std::chrono::microseconds(200)); - uint64_t epoch = 0; - mallctl("epoch", nullptr, nullptr, &epoch, sizeof(epoch)); - auto maybe_resident = getJemallocValue("stats.resident"); - if (!maybe_resident.has_value()) - continue; - - Int64 resident = *maybe_resident; - //LOG_INFO(getLogger("JEmalloc"), "Resident {}", ReadableSize(resident)); - MemoryTracker::setRSS(resident, false); - if (resident > total_memory_tracker.getHardLimit()) - purgeJemallocArenas(); - } -} - } void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log) @@ -905,11 +884,6 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - ThreadFromGlobalPool background_memory_thread([] - { - backgroundMemoryThread(); - }); - Poco::ThreadPool server_pool( /* minCapacity */3, /* maxCapacity */server_settings.max_connections, @@ -930,16 +904,7 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); - try - { - auto cgroups_reader = createCgroupsReader(); - global_context->setCgroupsReader(createCgroupsReader()); - } - catch (...) - { - if (server_settings.cgroups_memory_usage_observer_wait_time != 0) - tryLogCurrentException(log, "Failed to create cgroups reader"); - } + MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( @@ -1500,13 +1465,15 @@ try } std::optional cgroups_memory_usage_observer; - if (auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; wait_time != 0) + try { - auto cgroups_reader = global_context->getCgroupsReader(); - if (cgroups_reader) - cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time), std::move(cgroups_reader)); - else - LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); + auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; + if (wait_time != 0) + cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); + } + catch (Exception &) + { + tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); } std::string cert_path = config().getString("openSSL.server.certificateFile", ""); @@ -1602,8 +1569,6 @@ try background_memory_tracker.setDescription("(background)"); background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking); - total_memory_tracker.setAllowUseJemallocMemory(new_server_settings.allow_use_jemalloc_memory); - auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index b3e53c29d4a..a5c9875188b 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -57,12 +58,10 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_, - std::shared_ptr cgroups_reader_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : update_period(update_period_seconds) , log(getLogger("AsynchronousMetrics")) , protocol_server_metrics_func(protocol_server_metrics_func_) - , cgroups_reader(std::move(cgroups_reader_)) { #if defined(OS_LINUX) openFileIfExists("/proc/meminfo", meminfo); @@ -378,23 +377,13 @@ void AsynchronousMetrics::run() namespace { -uint64_t updateJemallocEpoch() -{ - uint64_t value = 0; - size_t size = sizeof(value); - mallctl("epoch", &value, &size, &value, size); - return value; -} - template Value saveJemallocMetricImpl( AsynchronousMetricValues & values, const std::string & jemalloc_full_name, const std::string & clickhouse_full_name) { - Value value{}; - size_t size = sizeof(value); - mallctl(jemalloc_full_name.c_str(), &value, &size, nullptr, 0); + auto value = getJemallocValue(jemalloc_full_name.c_str()); values[clickhouse_full_name] = AsynchronousMetricValue(value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html"); return value; } @@ -604,7 +593,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) // 'epoch' is a special mallctl -- it updates the statistics. Without it, all // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. - auto epoch = updateJemallocEpoch(); + auto epoch = getJemallocValue("epoch"); new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." }; // Collect the statistics themselves. diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 0b110f41fc3..bc379d4e92b 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -69,8 +69,7 @@ public: AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_, - std::shared_ptr cgroups_reader_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_); virtual ~AsynchronousMetrics(); diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index b12845df098..ab7ca69ca04 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -191,11 +191,9 @@ std::shared_ptr createCgroupsReader() } -CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_) - : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(std::move(cgroups_reader_)) -{ - cgroups_reader = createCgroupsReader(); -} +CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) + : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(createCgroupsReader()) +{} CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() { diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index 078307a6fa0..33e0f167a59 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -18,28 +18,20 @@ struct ICgroupsReader std::shared_ptr createCgroupsReader(); -/// Does two things: -/// 1. Periodically reads the memory usage of the process from Linux cgroups. -/// You can specify soft or hard memory limits: -/// - When the soft memory limit is hit, drop jemalloc cache. -/// - When the hard memory limit is hit, update MemoryTracking metric to throw memory exceptions faster. -/// The goal of this is to avoid that the process hits the maximum allowed memory limit at which there is a good -/// chance that the Limux OOM killer terminates it. All of this is done is because internal memory tracking in -/// ClickHouse can unfortunately under-estimate the actually used memory. -/// 2. Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). -/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server -/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' -/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.). -/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling -/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes -/// to the database. +/// Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). +/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server +/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' +/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.). +/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling +/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes +/// to the database. #if defined(OS_LINUX) class CgroupsMemoryUsageObserver { public: using OnMemoryAmountAvailableChangedFn = std::function; - explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_); + explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_); ~CgroupsMemoryUsageObserver(); void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_); diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 0c533711f78..629d039b483 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -28,28 +28,46 @@ void setJemallocValue(const char * name, T value) { T old_value; size_t old_value_size = sizeof(T); - if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) - { - LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); - return; - } - + mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T)); LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); } template -std::optional getJemallocValue(const char * name) +T getJemallocValue(const char * name) { T value; size_t value_size = sizeof(T); - if (mallctl(name, &value, &value_size, nullptr, 0)) - { - LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); - return std::nullopt; - } + mallctl(name, &value, &value_size, nullptr, 0); return value; } +template +struct JemallocMibCache +{ + explicit JemallocMibCache(const char * name) + { + mallctlnametomib(name, mib, &mib_length); + } + + void setValue(T value) + { + mallctlbymib(mib, mib_length, nullptr, nullptr, reinterpret_cast(&value), sizeof(T)); + } + + T getValue() + { + T value; + size_t value_size = sizeof(T); + mallctlbymib(mib, mib_length, &value, &value_size, nullptr, 0); + return value; + } + +private: + static constexpr size_t max_mib_length = 4; + size_t mib[max_mib_length]; + size_t mib_length = max_mib_length; +}; + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index a541eeeb25f..e237c3a0d33 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -108,8 +108,6 @@ void AllocationTrace::onFreeImpl(void * ptr, size_t size) const namespace ProfileEvents { extern const Event QueryMemoryLimitExceeded; - extern const Event MemoryAllocatorPurge; - extern const Event MemoryAllocatorPurgeTimeMicroseconds; } using namespace std::chrono_literals; @@ -119,8 +117,6 @@ static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; MemoryTracker total_memory_tracker(nullptr, VariableContext::Global); MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false); -std::atomic MemoryTracker::has_free_memory_in_allocator_arenas; - MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} @@ -500,12 +496,10 @@ void MemoryTracker::reset() } -void MemoryTracker::setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_) +void MemoryTracker::setRSS(Int64 rss_) { Int64 new_amount = rss_; - if (rss_) - total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); - has_free_memory_in_allocator_arenas.store(has_free_memory_in_allocator_arenas_, std::memory_order_relaxed); + total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 257ed7d0629..4085bb321ed 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -56,9 +56,6 @@ private: std::atomic soft_limit {0}; std::atomic hard_limit {0}; std::atomic profiler_limit {0}; - std::atomic_bool allow_use_jemalloc_memory {true}; - - static std::atomic has_free_memory_in_allocator_arenas; Int64 profiler_step = 0; @@ -153,14 +150,6 @@ public: { return soft_limit.load(std::memory_order_relaxed); } - void setAllowUseJemallocMemory(bool value) - { - allow_use_jemalloc_memory.store(value, std::memory_order_relaxed); - } - bool getAllowUseJemallocMmemory() const - { - return allow_use_jemalloc_memory.load(std::memory_order_relaxed); - } /** Set limit if it was not set. * Otherwise, set limit to new value, if new value is greater than previous limit. @@ -251,7 +240,7 @@ public: /// Reset current counter to an RSS value. /// Jemalloc may have pre-allocated arenas, they are accounted in RSS. /// We can free this arenas in case of exception to avoid OOM. - static void setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_); + static void setRSS(Int64 rss_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp new file mode 100644 index 00000000000..dce47b83667 --- /dev/null +++ b/src/Common/MemoryWorker.cpp @@ -0,0 +1,49 @@ +#include "Common/ThreadPool.h" +#include + +#include +#include + +namespace DB +{ + +#if USE_JEMALLOC +MemoryWorker::MemoryWorker(uint64_t period_ms_) + : period_ms(period_ms_) +{ + background_thread = ThreadFromGlobalPool([this] { backgroundThread(); }); +} + +MemoryWorker::~MemoryWorker() +{ + { + std::unique_lock lock(mutex); + shutdown = true; + } + cv.notify_all(); + + if (background_thread.joinable()) + background_thread.join(); +} + +void MemoryWorker::backgroundThread() +{ + JemallocMibCache epoch_mib("epoch"); + JemallocMibCache resident_mib("stats.resident"); + std::unique_lock lock(mutex); + while (true) + { + cv.wait_for(lock, period_ms, [this] { return shutdown; }); + if (shutdown) + return; + + epoch_mib.setValue(0); + Int64 resident = resident_mib.getValue(); + MemoryTracker::setRSS(resident); + if (resident > total_memory_tracker.getHardLimit()) + purgeJemallocArenas(); + } +} +#endif + +} diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h new file mode 100644 index 00000000000..8048194d9cd --- /dev/null +++ b/src/Common/MemoryWorker.h @@ -0,0 +1,34 @@ +#pragma once + +#include + +#include "config.h" + +namespace DB +{ + +#if USE_JEMALLOC +class MemoryWorker +{ +public: + explicit MemoryWorker(uint64_t period_ms_); + + ~MemoryWorker(); +private: + void backgroundThread(); + + ThreadFromGlobalPool background_thread; + + std::mutex mutex; + std::condition_variable cv; + bool shutdown = false; + + std::chrono::milliseconds period_ms; +}; +#else +class MemoryWorker +{ +}; +#endif + +} diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 3e404b7152b..86166ffe31b 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -115,7 +115,7 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, context_->getCgroupsReader()), context(std::move(context_)) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) { } diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 28b32a6e6a5..aaea0388239 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -157,6 +157,7 @@ namespace DB M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ + M(UInt64, memory_worker_period_ms, 100, "Period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage.", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6a24e049998..f70ccfd77be 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -406,8 +406,6 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr cluster_discovery TSA_GUARDED_BY(clusters_mutex); size_t clusters_version TSA_GUARDED_BY(clusters_mutex) = 0; - std::shared_ptr cgroups_reader; - /// No lock required for async_insert_queue modified only during initialization std::shared_ptr async_insert_queue; @@ -5631,16 +5629,6 @@ const ServerSettings & Context::getServerSettings() const return shared->server_settings; } -void Context::setCgroupsReader(std::shared_ptr cgroups_reader_) -{ - shared->cgroups_reader = std::move(cgroups_reader_); -} - -std::shared_ptr Context::getCgroupsReader() const -{ - return shared->cgroups_reader; -} - uint64_t HTTPContext::getMaxHstsAge() const { return context->getSettingsRef().hsts_max_age; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 33b742d20ad..0e3c0591c12 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1344,9 +1344,6 @@ public: const ServerSettings & getServerSettings() const; - void setCgroupsReader(std::shared_ptr cgroups_reader_); - std::shared_ptr getCgroupsReader() const; - private: std::shared_ptr getSettingsConstraintsAndCurrentProfilesWithLock() const; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 6ee0168bede..872a9f864df 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -57,7 +57,7 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( unsigned heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : WithContext(global_context_) - , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, getContext()->getCgroupsReader()) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index d242b6de4ec..ee99c472620 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -63,7 +63,6 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context /// current setting values, one needs to ask the components directly. std::unordered_map> changeable_settings = { {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, - {"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), ChangeableWithoutRestart::Yes}}, {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, diff --git a/tests/integration/test_memory_limit/test.py b/tests/integration/test_memory_limit/test.py index 6d6745711da..db68a38c1b1 100644 --- a/tests/integration/test_memory_limit/test.py +++ b/tests/integration/test_memory_limit/test.py @@ -13,7 +13,6 @@ node = cluster.add_instance( "configs/async_metrics_no.xml", ], mem_limit="4g", - env_variables={"MALLOC_CONF": "dirty_decay_ms:0"}, ) From 9ec1fd1ab769b2f6c6ad713b4e747a89bde48b78 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 11:29:03 +0200 Subject: [PATCH 04/40] Fix non-jemalloc builds --- src/Common/MemoryWorker.cpp | 2 +- src/Common/MemoryWorker.h | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index dce47b83667..e5ebbe3b979 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -1,8 +1,8 @@ -#include "Common/ThreadPool.h" #include #include #include +#include namespace DB { diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 8048194d9cd..5f02fd0b1d0 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -28,6 +28,8 @@ private: #else class MemoryWorker { +public: + explicit MemoryWorker(uint64_t /*period_ms_*/) {} }; #endif From 05c7dc582a48d738bed82bcb24d3a7619fec8bc9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 13:40:03 +0200 Subject: [PATCH 05/40] Add some comments --- src/Common/Jemalloc.h | 3 +++ src/Common/MemoryWorker.h | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 629d039b483..dfa265c5e59 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -41,6 +41,9 @@ T getJemallocValue(const char * name) return value; } +/// Each mallctl call consists of string name lookup which can be expensive. +/// This can be avoided by translating name to "Management Information Base" (MIB) +/// and using it in mallctlbymib calls template struct JemallocMibCache { diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 5f02fd0b1d0..6c0a578aa61 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -8,6 +8,12 @@ namespace DB { #if USE_JEMALLOC +/// Correct MemoryTracker based on stats.resident read from jemalloc. +/// This requires jemalloc built with --enable-stats which we use. +/// The worker spawns a background thread which moves the jemalloc epoch (updates internal stats), +/// and fetches the current stats.resident whose value is sent to global MemoryTracker. +/// Additionally, if the current memory usage is higher than global hard limit, +/// jemalloc's dirty pages are forcefully purged. class MemoryWorker { public: From 7d66f400b25a5bf0f2f43f06c6ed432d1c572fb6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 16:51:49 +0200 Subject: [PATCH 06/40] Better --- src/Common/Jemalloc.h | 5 +++++ src/Common/MemoryTracker.cpp | 13 ++++++++----- src/Common/MemoryTracker.h | 8 ++++---- src/Common/MemoryWorker.cpp | 27 +++++++++++++++++++++++++-- src/Common/ProfileEvents.cpp | 3 +++ 5 files changed, 45 insertions(+), 11 deletions(-) diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index dfa265c5e59..22a94a44eba 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -65,6 +65,11 @@ struct JemallocMibCache return value; } + void run() + { + mallctlbymib(mib, mib_length, nullptr, nullptr, nullptr, 0); + } + private: static constexpr size_t max_mib_length = 4; size_t mib[max_mib_length]; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index e237c3a0d33..49a3a6ef7ef 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -242,6 +242,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed * So, we allow over-allocations. */ Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); + Int64 will_be_rss = size + rss.load(std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end() && size) @@ -290,7 +291,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } - if (unlikely(current_hard_limit && will_be > current_hard_limit)) + if (unlikely(current_hard_limit && (will_be > current_hard_limit || will_be_rss > current_hard_limit))) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { @@ -310,12 +311,13 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, "Memory limit{}{} exceeded: " - "would use {} (attempt to allocate chunk of {} bytes), maximum: {}." + "would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}." "{}{}", description ? " " : "", description ? description : "", formatReadableSizeWithBinarySuffix(will_be), size, + formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)), formatReadableSizeWithBinarySuffix(current_hard_limit), overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ", toDescription(overcommit_result)); @@ -496,17 +498,18 @@ void MemoryTracker::reset() } -void MemoryTracker::setRSS(Int64 rss_) +void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_) { - Int64 new_amount = rss_; + Int64 new_amount = allocated_; total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); + total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::set(metric_loaded, new_amount); bool log_memory_usage = true; - total_memory_tracker.updatePeak(rss_, log_memory_usage); + total_memory_tracker.updatePeak(new_amount, log_memory_usage); } diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 4085bb321ed..add8bcb43d2 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -57,6 +57,8 @@ private: std::atomic hard_limit {0}; std::atomic profiler_limit {0}; + std::atomic rss{0}; + Int64 profiler_step = 0; /// To test exception safety of calling code, memory tracker throws an exception on each memory allocation with specified probability. @@ -237,10 +239,8 @@ public: /// Reset the accumulated data. void reset(); - /// Reset current counter to an RSS value. - /// Jemalloc may have pre-allocated arenas, they are accounted in RSS. - /// We can free this arenas in case of exception to avoid OOM. - static void setRSS(Int64 rss_); + /// update values based on external information (e.g. jemalloc's stat) + static void updateValues(Int64 rss_, Int64 allocated_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index e5ebbe3b979..ae488a47b67 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -3,11 +3,23 @@ #include #include #include +#include + +namespace ProfileEvents +{ + extern const Event MemoryAllocatorPurge; + extern const Event MemoryAllocatorPurgeTimeMicroseconds; + extern const Event MemoryWorkerRun; + extern const Event MemoryWorkerRunElapsedMicroseconds; +} namespace DB { #if USE_JEMALLOC +#define STRINGIFY_HELPER(x) #x +#define STRINGIFY(x) STRINGIFY_HELPER(x) + MemoryWorker::MemoryWorker(uint64_t period_ms_) : period_ms(period_ms_) { @@ -30,6 +42,8 @@ void MemoryWorker::backgroundThread() { JemallocMibCache epoch_mib("epoch"); JemallocMibCache resident_mib("stats.resident"); + JemallocMibCache allocated_mib("stats.allocated"); + JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); std::unique_lock lock(mutex); while (true) { @@ -37,11 +51,20 @@ void MemoryWorker::backgroundThread() if (shutdown) return; + Stopwatch total_watch; epoch_mib.setValue(0); Int64 resident = resident_mib.getValue(); - MemoryTracker::setRSS(resident); if (resident > total_memory_tracker.getHardLimit()) - purgeJemallocArenas(); + { + Stopwatch purge_watch; + purge_mib.run(); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); + } + + MemoryTracker::updateValues(resident, allocated_mib.getValue()); + ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); + ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); } } #endif diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 871ba7cab8b..d85c21fcded 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -778,6 +778,9 @@ The server successfully detected this situation and will download merged part fr M(GWPAsanAllocateSuccess, "Number of successful allocations done by GWPAsan") \ M(GWPAsanAllocateFailed, "Number of failed allocations done by GWPAsan (i.e. filled pool)") \ M(GWPAsanFree, "Number of free operations done by GWPAsan") \ + \ + M(MemoryWorkerRun, "Number of runs done by MemoryWorker in background") \ + M(MemoryWorkerRunElapsedMicroseconds, "Total time spent by MemoryWorker for background work") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS From 2147a96475717f0af53dd62f487c011a5b9b933a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 21 Jul 2024 11:32:57 +0200 Subject: [PATCH 07/40] Better --- src/Common/AsynchronousMetrics.cpp | 4 +++ src/Common/MemoryTracker.cpp | 31 +++++++++++++++---- src/Common/MemoryTracker.h | 2 +- src/Common/MemoryWorker.cpp | 7 ++++- src/Coordination/KeeperDispatcher.cpp | 8 ++++- .../configs/keeper_config2.xml | 2 +- .../configs/keeper_config3.xml | 2 +- 7 files changed, 45 insertions(+), 11 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index a5c9875188b..dc2f687004b 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -638,6 +638,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) "The amount of virtual memory mapped for the use of stack and for the allocated memory, in bytes." " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; + +#if !USE_JEMALLOC + MemoryTracker::updateValues(data.resident, data.resident, /*force_update=*/true); +#endif } { diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 49a3a6ef7ef..07d6ba98745 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -221,6 +221,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// For global memory tracker always update memory usage. amount.fetch_add(size, std::memory_order_relaxed); + rss.fetch_add(size, std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) @@ -242,7 +243,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed * So, we allow over-allocations. */ Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); - Int64 will_be_rss = size + rss.load(std::memory_order_relaxed); + Int64 will_be_rss = size ? size + rss.fetch_add(size, std::memory_order_relaxed) : rss.load(std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end() && size) @@ -269,6 +270,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// Revert amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); @@ -291,7 +293,8 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } - if (unlikely(current_hard_limit && (will_be > current_hard_limit || will_be_rss > current_hard_limit))) + if (unlikely( + current_hard_limit && (will_be > current_hard_limit || (level == VariableContext::Global && will_be_rss > current_hard_limit)))) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { @@ -303,6 +306,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// Revert amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); @@ -411,6 +415,7 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability) { /// For global memory tracker always update memory usage. amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::sub(metric_loaded, size); @@ -424,7 +429,12 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability) } Int64 accounted_size = size; - if (level == VariableContext::Thread || level == VariableContext::Global) + if (level == VariableContext::Global) + { + amount.fetch_sub(accounted_size, std::memory_order_relaxed); + rss.fetch_sub(accounted_size, std::memory_order_relaxed); + } + else if (level == VariableContext::Thread) { /// Could become negative if memory allocated in this thread is freed in another one amount.fetch_sub(accounted_size, std::memory_order_relaxed); @@ -498,12 +508,21 @@ void MemoryTracker::reset() } -void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_) +void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_, bool force_update) { - Int64 new_amount = allocated_; - total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); + if (likely(!force_update && total_memory_tracker.amount.load(std::memory_order_relaxed) >= 0)) + return; + + Int64 new_amount = allocated_; + LOG_INFO( + getLogger("MemoryTracker"), + "Correcting the value of global memory tracker from {} to {}", + ReadableSize(total_memory_tracker.amount.load(std::memory_order_relaxed)), + ReadableSize(allocated_)); + total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); + auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::set(metric_loaded, new_amount); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index add8bcb43d2..4913be9781f 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -240,7 +240,7 @@ public: void reset(); /// update values based on external information (e.g. jemalloc's stat) - static void updateValues(Int64 rss_, Int64 allocated_); + static void updateValues(Int64 rss_, Int64 allocated_, bool force_update); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index ae488a47b67..23cd90178ff 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -44,6 +44,7 @@ void MemoryWorker::backgroundThread() JemallocMibCache resident_mib("stats.resident"); JemallocMibCache allocated_mib("stats.allocated"); JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); + bool first_run = false; std::unique_lock lock(mutex); while (true) { @@ -62,9 +63,13 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } - MemoryTracker::updateValues(resident, allocated_mib.getValue()); + /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed + /// during initialization + MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); + + first_run = false; } } #endif diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 8c7e6405153..332662117d8 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -148,7 +148,13 @@ void KeeperDispatcher::requestThread() Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit(); if (configuration_and_settings->standalone_keeper && isExceedingMemorySoftLimit() && checkIfRequestIncreaseMem(request.request)) { - LOG_WARNING(log, "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type is {}", ReadableSize(mem_soft_limit), ReadableSize(total_memory_tracker.get()), request.request->getOpNum()); + LOG_WARNING( + log, + "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type " + "is {}", + ReadableSize(mem_soft_limit), + ReadableSize(total_memory_tracker.get()), + request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; } diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml index 25ececea3e8..e71b93379d0 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml @@ -16,7 +16,7 @@ az-zoo2 1 - 20000000 + 200000000 10000 diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml index 81e343b77c9..cf4a4686f2c 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml @@ -13,7 +13,7 @@ 2181 3 - 20000000 + 200000000 10000 From 1c3f7d0fd0fbf27692ff29d8309382eae7b7a598 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 14:58:00 +0200 Subject: [PATCH 08/40] Small fix --- programs/server/Server.cpp | 4 ++-- src/Common/MemoryWorker.cpp | 15 ++++++++++----- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 04480f0bfe9..5691d82e216 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -904,8 +904,6 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); - MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); - /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -1198,6 +1196,8 @@ try FailPointInjection::enableFromGlobalConfig(config()); + MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); + int default_oom_score = 0; #if !defined(NDEBUG) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 23cd90178ff..2b945a30d3d 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -2,8 +2,9 @@ #include #include -#include #include +#include +#include namespace ProfileEvents { @@ -23,6 +24,7 @@ namespace DB MemoryWorker::MemoryWorker(uint64_t period_ms_) : period_ms(period_ms_) { + LOG_INFO(getLogger("MemoryWorker"), "Starting background memory thread with period of {}ms", period_ms.count()); background_thread = ThreadFromGlobalPool([this] { backgroundThread(); }); } @@ -42,9 +44,10 @@ void MemoryWorker::backgroundThread() { JemallocMibCache epoch_mib("epoch"); JemallocMibCache resident_mib("stats.resident"); + JemallocMibCache active_mib("stats.active"); JemallocMibCache allocated_mib("stats.allocated"); JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); - bool first_run = false; + bool first_run = true; std::unique_lock lock(mutex); while (true) { @@ -55,6 +58,11 @@ void MemoryWorker::backgroundThread() Stopwatch total_watch; epoch_mib.setValue(0); Int64 resident = resident_mib.getValue(); + + /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed + /// during initialization + MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); + if (resident > total_memory_tracker.getHardLimit()) { Stopwatch purge_watch; @@ -63,9 +71,6 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } - /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed - /// during initialization - MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); From d78cfd030fa8364456ac5283a6a1469703c53b40 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 21:47:46 +0200 Subject: [PATCH 09/40] Use cgroups as source --- programs/keeper/Keeper.cpp | 8 +- programs/server/Server.cpp | 9 +- src/Common/AsynchronousMetrics.cpp | 32 +- src/Common/AsynchronousMetrics.h | 10 +- src/Common/CgroupsMemoryUsageObserver.cpp | 168 +---------- src/Common/CgroupsMemoryUsageObserver.h | 12 - src/Common/MemoryTracker.cpp | 9 +- src/Common/MemoryTracker.h | 3 +- src/Common/MemoryWorker.cpp | 280 ++++++++++++++++-- src/Common/MemoryWorker.h | 49 ++- .../KeeperAsynchronousMetrics.cpp | 9 +- src/Coordination/KeeperAsynchronousMetrics.h | 8 +- src/Core/ServerSettings.h | 2 +- .../ServerAsynchronousMetrics.cpp | 6 +- src/Interpreters/ServerAsynchronousMetrics.h | 5 +- 15 files changed, 367 insertions(+), 243 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index b10d3f34623..d308e741311 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -376,7 +376,8 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); - MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 100)); + MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 0)); + memory_worker.start(); static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); @@ -419,8 +420,9 @@ try for (const auto & server : *servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; - } - ); + }, + /*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc, + /*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None); std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5691d82e216..1fc1df1494c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -904,6 +904,8 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); + /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -922,8 +924,9 @@ try for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; - } - ); + }, + /*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc, + /*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None); /// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown() /// Otherwise GlobalThreadPool::shutdown() will hang, since Context holds some threads. @@ -1196,7 +1199,7 @@ try FailPointInjection::enableFromGlobalConfig(config()); - MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); + memory_worker.start(); int default_oom_score = 0; diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index dc2f687004b..53b8e13eaaa 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -58,10 +58,14 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) : update_period(update_period_seconds) , log(getLogger("AsynchronousMetrics")) , protocol_server_metrics_func(protocol_server_metrics_func_) + , update_jemalloc_epoch(update_jemalloc_epoch_) + , update_rss(update_rss_) { #if defined(OS_LINUX) openFileIfExists("/proc/meminfo", meminfo); @@ -377,6 +381,14 @@ void AsynchronousMetrics::run() namespace { +uint64_t updateJemallocEpoch() +{ + uint64_t value = 0; + size_t size = sizeof(value); + mallctl("epoch", &value, &size, &value, size); + return value; +} + template Value saveJemallocMetricImpl( AsynchronousMetricValues & values, @@ -593,8 +605,11 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) // 'epoch' is a special mallctl -- it updates the statistics. Without it, all // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. - auto epoch = getJemallocValue("epoch"); - new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." }; + auto epoch = update_jemalloc_epoch ? updateJemallocEpoch() : getJemallocValue("epoch"); + new_values["jemalloc.epoch"] + = {epoch, + "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other " + "`jemalloc` metrics."}; // Collect the statistics themselves. saveJemallocMetric(new_values, "allocated"); @@ -607,10 +622,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) saveJemallocMetric(new_values, "background_thread.num_threads"); saveJemallocMetric(new_values, "background_thread.num_runs"); saveJemallocMetric(new_values, "background_thread.run_intervals"); - saveJemallocProf(new_values, "active"); + saveJemallocProf(new_values, "active"); saveAllArenasMetric(new_values, "pactive"); - [[maybe_unused]] size_t je_malloc_pdirty = saveAllArenasMetric(new_values, "pdirty"); - [[maybe_unused]] size_t je_malloc_pmuzzy = saveAllArenasMetric(new_values, "pmuzzy"); + saveAllArenasMetric(new_values, "pdirty"); + saveAllArenasMetric(new_values, "pmuzzy"); saveAllArenasMetric(new_values, "dirty_purged"); saveAllArenasMetric(new_values, "muzzy_purged"); #endif @@ -639,9 +654,8 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; -#if !USE_JEMALLOC - MemoryTracker::updateValues(data.resident, data.resident, /*force_update=*/true); -#endif + if (update_rss) + MemoryTracker::updateRSS(data.resident); } { diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index bc379d4e92b..eb6ede7a558 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -7,10 +7,8 @@ #include #include -#include #include #include -#include #include #include #include @@ -69,7 +67,9 @@ public: AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); virtual ~AsynchronousMetrics(); @@ -92,7 +92,6 @@ private: virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; - std::shared_ptr cgroups_reader; std::unique_ptr thread; @@ -113,6 +112,9 @@ private: MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex); #endif + const bool update_jemalloc_epoch; + const bool update_rss; + #if defined(OS_LINUX) std::optional meminfo TSA_GUARDED_BY(data_mutex); std::optional loadavg TSA_GUARDED_BY(data_mutex); diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index ab7ca69ca04..afeac1808b2 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -13,12 +13,8 @@ #include #include -#include -#include -#include using namespace DB; -namespace fs = std::filesystem; namespace DB { @@ -29,170 +25,8 @@ extern const int FILE_DOESNT_EXIST; extern const int INCORRECT_DATA; } -} - -namespace -{ - -/// Format is -/// kernel 5 -/// rss 15 -/// [...] -uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) -{ - while (!buf.eof()) - { - std::string current_key; - readStringUntilWhitespace(current_key, buf); - if (current_key != key) - { - std::string dummy; - readStringUntilNewlineInto(dummy, buf); - buf.ignore(); - continue; - } - - assertChar(' ', buf); - uint64_t value = 0; - readIntText(value, buf); - return value; - } - - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find '{}' in '{}'", key, buf.getFileName()); -} - -struct CgroupsV1Reader : ICgroupsReader -{ - explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } - - uint64_t readMemoryUsage() override - { - std::lock_guard lock(mutex); - buf.rewind(); - return readMetricFromStatFile(buf, "rss"); - } - -private: - std::mutex mutex; - ReadBufferFromFile buf TSA_GUARDED_BY(mutex); -}; - -struct CgroupsV2Reader : ICgroupsReader -{ - explicit CgroupsV2Reader(const fs::path & stat_file_dir) - : current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat") - { - } - - uint64_t readMemoryUsage() override - { - std::lock_guard lock(mutex); - current_buf.rewind(); - stat_buf.rewind(); - - int64_t mem_usage = 0; - /// memory.current contains a single number - /// the reason why we subtract it described here: https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 - readIntText(mem_usage, current_buf); - mem_usage -= readMetricFromStatFile(stat_buf, "inactive_file"); - chassert(mem_usage >= 0, "Negative memory usage"); - return mem_usage; - } - -private: - std::mutex mutex; - ReadBufferFromFile current_buf TSA_GUARDED_BY(mutex); - ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex); -}; - -/// Caveats: -/// - All of the logic in this file assumes that the current process is the only process in the -/// containing cgroup (or more precisely: the only process with significant memory consumption). -/// If this is not the case, then other processe's memory consumption may affect the internal -/// memory tracker ... -/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a -/// decade and will go away at some point, hierarchical detection is only implemented for v2. -/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such -/// systems existed only for a short transition period. - -std::optional getCgroupsV2Path() -{ - if (!cgroupsV2Enabled()) - return {}; - - if (!cgroupsV2MemoryControllerEnabled()) - return {}; - - fs::path current_cgroup = cgroupV2PathOfProcess(); - if (current_cgroup.empty()) - return {}; - - /// Return the bottom-most nested current memory file. If there is no such file at the current - /// level, try again at the parent level as memory settings are inherited. - while (current_cgroup != default_cgroups_mount.parent_path()) - { - const auto current_path = current_cgroup / "memory.current"; - const auto stat_path = current_cgroup / "memory.stat"; - if (fs::exists(current_path) && fs::exists(stat_path)) - return {current_cgroup}; - current_cgroup = current_cgroup.parent_path(); - } - return {}; -} - -std::optional getCgroupsV1Path() -{ - auto path = default_cgroups_mount / "memory/memory.stat"; - if (!fs::exists(path)) - return {}; - return {default_cgroups_mount / "memory"}; -} - -enum class CgroupsVersion : uint8_t -{ - V1, - V2 -}; - -std::pair getCgroupsPath() -{ - auto v2_path = getCgroupsV2Path(); - if (v2_path.has_value()) - return {*v2_path, CgroupsVersion::V2}; - - auto v1_path = getCgroupsV1Path(); - if (v1_path.has_value()) - return {*v1_path, CgroupsVersion::V1}; - - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); -} - -} - -namespace DB -{ - -std::shared_ptr createCgroupsReader() -{ - const auto [cgroup_path, version] = getCgroupsPath(); - LOG_INFO( - getLogger("CgroupsReader"), - "Will create cgroup reader from '{}' (cgroups version: {})", - cgroup_path, - (version == CgroupsVersion::V1) ? "v1" : "v2"); - - if (version == CgroupsVersion::V2) - return std::make_shared(cgroup_path); - else - { - chassert(version == CgroupsVersion::V1); - return std::make_shared(cgroup_path); - } - -} - CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) - : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(createCgroupsReader()) + : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) {} CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index 33e0f167a59..3de83d6b437 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -3,21 +3,11 @@ #include #include -#include #include namespace DB { -struct ICgroupsReader -{ - virtual ~ICgroupsReader() = default; - - virtual uint64_t readMemoryUsage() = 0; -}; - -std::shared_ptr createCgroupsReader(); - /// Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). /// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server /// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' @@ -54,8 +44,6 @@ private: void runThread(); - std::shared_ptr cgroups_reader; - std::mutex thread_mutex; std::condition_variable cond; ThreadFromGlobalPool thread; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 07d6ba98745..0ffae89ffa6 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -508,13 +508,13 @@ void MemoryTracker::reset() } -void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_, bool force_update) +void MemoryTracker::updateRSS(Int64 rss_) { total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); +} - if (likely(!force_update && total_memory_tracker.amount.load(std::memory_order_relaxed) >= 0)) - return; - +void MemoryTracker::updateAllocated(Int64 allocated_) +{ Int64 new_amount = allocated_; LOG_INFO( getLogger("MemoryTracker"), @@ -531,7 +531,6 @@ void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_, bool force_update total_memory_tracker.updatePeak(new_amount, log_memory_usage); } - void MemoryTracker::setSoftLimit(Int64 value) { soft_limit.store(value, std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 4913be9781f..d2db8489f19 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -240,7 +240,8 @@ public: void reset(); /// update values based on external information (e.g. jemalloc's stat) - static void updateValues(Int64 rss_, Int64 allocated_, bool force_update); + static void updateRSS(Int64 rss_); + static void updateAllocated(Int64 allocated_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 2b945a30d3d..42e797a80d6 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -1,11 +1,21 @@ #include +#include +#include +#include +#include #include #include #include #include #include +#include +#include +#include + +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event MemoryAllocatorPurge; @@ -17,14 +27,227 @@ namespace ProfileEvents namespace DB { -#if USE_JEMALLOC -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) - -MemoryWorker::MemoryWorker(uint64_t period_ms_) - : period_ms(period_ms_) +namespace ErrorCodes { - LOG_INFO(getLogger("MemoryWorker"), "Starting background memory thread with period of {}ms", period_ms.count()); + extern const int FILE_DOESNT_EXIST; + extern const int INCORRECT_DATA; +} + +#if defined(OS_LINUX) +struct ICgroupsReader +{ + virtual ~ICgroupsReader() = default; + + virtual uint64_t readMemoryUsage() = 0; +}; + +namespace +{ + +/// Format is +/// kernel 5 +/// rss 15 +/// [...] +uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) +{ + while (!buf.eof()) + { + std::string current_key; + readStringUntilWhitespace(current_key, buf); + if (current_key != key) + { + std::string dummy; + readStringUntilNewlineInto(dummy, buf); + buf.ignore(); + continue; + } + + assertChar(' ', buf); + uint64_t value = 0; + readIntText(value, buf); + return value; + } + + LOG_ERROR(getLogger("CgroupsReader"), "Cannot find '{}' in '{}'", key, buf.getFileName()); + return 0; +} + +struct CgroupsV1Reader : ICgroupsReader +{ + explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + buf.rewind(); + return readMetricFromStatFile(buf, "rss"); + } + +private: + std::mutex mutex; + ReadBufferFromFile buf TSA_GUARDED_BY(mutex); +}; + +struct CgroupsV2Reader : ICgroupsReader +{ + explicit CgroupsV2Reader(const fs::path & stat_file_dir) : stat_buf(stat_file_dir / "memory.stat") { } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + stat_buf.rewind(); + return readMetricFromStatFile(stat_buf, "anon"); + } + +private: + std::mutex mutex; + ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex); +}; + +/// Caveats: +/// - All of the logic in this file assumes that the current process is the only process in the +/// containing cgroup (or more precisely: the only process with significant memory consumption). +/// If this is not the case, then other processe's memory consumption may affect the internal +/// memory tracker ... +/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a +/// decade and will go away at some point, hierarchical detection is only implemented for v2. +/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such +/// systems existed only for a short transition period. + +std::optional getCgroupsV2Path() +{ + if (!cgroupsV2Enabled()) + return {}; + + if (!cgroupsV2MemoryControllerEnabled()) + return {}; + + fs::path current_cgroup = cgroupV2PathOfProcess(); + if (current_cgroup.empty()) + return {}; + + /// Return the bottom-most nested current memory file. If there is no such file at the current + /// level, try again at the parent level as memory settings are inherited. + while (current_cgroup != default_cgroups_mount.parent_path()) + { + const auto current_path = current_cgroup / "memory.current"; + const auto stat_path = current_cgroup / "memory.stat"; + if (fs::exists(current_path) && fs::exists(stat_path)) + return {current_cgroup}; + current_cgroup = current_cgroup.parent_path(); + } + return {}; +} + +std::optional getCgroupsV1Path() +{ + auto path = default_cgroups_mount / "memory/memory.stat"; + if (!fs::exists(path)) + return {}; + return {default_cgroups_mount / "memory"}; +} + +enum class CgroupsVersion : uint8_t +{ + V1, + V2 +}; + +std::pair getCgroupsPath() +{ + auto v2_path = getCgroupsV2Path(); + if (v2_path.has_value()) + return {*v2_path, CgroupsVersion::V2}; + + auto v1_path = getCgroupsV1Path(); + if (v1_path.has_value()) + return {*v1_path, CgroupsVersion::V1}; + + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); +} + +std::shared_ptr createCgroupsReader() +{ + const auto [cgroup_path, version] = getCgroupsPath(); + LOG_INFO( + getLogger("CgroupsReader"), + "Will create cgroup reader from '{}' (cgroups version: {})", + cgroup_path, + (version == CgroupsVersion::V1) ? "v1" : "v2"); + + if (version == CgroupsVersion::V2) + return std::make_shared(cgroup_path); + else + { + chassert(version == CgroupsVersion::V1); + return std::make_shared(cgroup_path); + } + +} +#endif + +constexpr uint64_t cgroups_memory_usage_tick_ms{50}; +constexpr uint64_t jemalloc_memory_usage_tick_ms{100}; + +std::string_view sourceToString(MemoryWorker::MemoryUsageSource source) +{ + switch (source) + { + case MemoryWorker::MemoryUsageSource::Cgroups: return "Cgroups"; + case MemoryWorker::MemoryUsageSource::Jemalloc: return "Jemalloc"; + case MemoryWorker::MemoryUsageSource::None: return "None"; + } +} + +} + +/// We try to pick the best possible supported source for reading memory usage. +/// Supported sources in order of priority +/// - reading from cgroups' pseudo-files (fastest and most accurate) +/// - reading jemalloc's resident stat (doesn't take into account allocations that didn't use jemalloc) +/// Also, different tick rates are used because not all options are equally fast +MemoryWorker::MemoryWorker(uint64_t period_ms_) + : log(getLogger("MemoryWorker")) + , period_ms(period_ms_) +{ +#if defined(OS_LINUX) + try + { + cgroups_reader = createCgroupsReader(); + source = MemoryUsageSource::Cgroups; + if (period_ms == 0) + period_ms = cgroups_memory_usage_tick_ms; + + return; + } + catch (...) + { + tryLogCurrentException(log, "Cannot use cgroups reader"); + } +#endif + +#if USE_JEMALLOC + source = MemoryUsageSource::Jemalloc; + if (period_ms == 0) + period_ms = jemalloc_memory_usage_tick_ms; +#endif +} + +MemoryWorker::MemoryUsageSource MemoryWorker::getSource() +{ + return source; +} + +void MemoryWorker::start() +{ + if (source == MemoryUsageSource::None) + return; + + LOG_INFO( + getLogger("MemoryWorker"), + "Starting background memory thread with period of {}ms, using {} as source", + period_ms, + sourceToString(source)); background_thread = ThreadFromGlobalPool([this] { backgroundThread(); }); } @@ -40,29 +263,39 @@ MemoryWorker::~MemoryWorker() background_thread.join(); } +uint64_t MemoryWorker::getMemoryUsage() +{ + switch (source) + { + case MemoryUsageSource::Cgroups: + return cgroups_reader->readMemoryUsage(); + case MemoryUsageSource::Jemalloc: + return resident_mib.getValue(); + case MemoryUsageSource::None: + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to fetch memory usage while no memory source can be used"); + } +} + void MemoryWorker::backgroundThread() { - JemallocMibCache epoch_mib("epoch"); - JemallocMibCache resident_mib("stats.resident"); - JemallocMibCache active_mib("stats.active"); - JemallocMibCache allocated_mib("stats.allocated"); - JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); + std::chrono::milliseconds chrono_period_ms{period_ms}; bool first_run = true; std::unique_lock lock(mutex); while (true) { - cv.wait_for(lock, period_ms, [this] { return shutdown; }); + cv.wait_for(lock, chrono_period_ms, [this] { return shutdown; }); if (shutdown) return; Stopwatch total_watch; - epoch_mib.setValue(0); - Int64 resident = resident_mib.getValue(); - /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed - /// during initialization - MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); + if (source == MemoryUsageSource::Jemalloc) + epoch_mib.setValue(0); + Int64 resident = getMemoryUsage(); + MemoryTracker::updateRSS(resident); + +#if USE_JEMALLOC if (resident > total_memory_tracker.getHardLimit()) { Stopwatch purge_watch; @@ -71,12 +304,19 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } + if (unlikely(first_run || total_memory_tracker.get() < 0)) + { + if (source != MemoryUsageSource::Jemalloc) + epoch_mib.setValue(0); + + MemoryTracker::updateAllocated(allocated_mib.getValue()); + } +#endif + ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); - first_run = false; } } -#endif } diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 6c0a578aa61..6fde93d63ad 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -1,13 +1,14 @@ #pragma once +#include #include - -#include "config.h" +#include namespace DB { -#if USE_JEMALLOC +struct ICgroupsReader; + /// Correct MemoryTracker based on stats.resident read from jemalloc. /// This requires jemalloc built with --enable-stats which we use. /// The worker spawns a background thread which moves the jemalloc epoch (updates internal stats), @@ -19,8 +20,21 @@ class MemoryWorker public: explicit MemoryWorker(uint64_t period_ms_); + enum class MemoryUsageSource : uint8_t + { + None, + Cgroups, + Jemalloc + }; + + MemoryUsageSource getSource(); + + void start(); + ~MemoryWorker(); private: + uint64_t getMemoryUsage(); + void backgroundThread(); ThreadFromGlobalPool background_thread; @@ -29,14 +43,27 @@ private: std::condition_variable cv; bool shutdown = false; - std::chrono::milliseconds period_ms; -}; -#else -class MemoryWorker -{ -public: - explicit MemoryWorker(uint64_t /*period_ms_*/) {} -}; + LoggerPtr log; + + uint64_t period_ms; + + MemoryUsageSource source{MemoryUsageSource::None}; + +#if defined(OS_LINUX) + std::shared_ptr cgroups_reader; #endif +#if USE_JEMALLOC + JemallocMibCache epoch_mib{"epoch"}; + JemallocMibCache resident_mib{"stats.resident"}; + JemallocMibCache allocated_mib{"stats.allocated"}; + +#define STRINGIFY_HELPER(x) #x +#define STRINGIFY(x) STRINGIFY_HELPER(x) + JemallocMibCache purge_mib{"arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"}; +#undef STRINGIFY +#undef STRINGIFY_HELPER +#endif +}; + } diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 86166ffe31b..157858f3c44 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -114,8 +114,13 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM } KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( - ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) + ContextPtr context_, + unsigned update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_) + , context(std::move(context_)) { } diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index ec0e60cbb6e..a2ab7cab756 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -13,9 +13,13 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics { public: KeeperAsynchronousMetrics( - ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); - ~KeeperAsynchronousMetrics() override; + ContextPtr context_, + unsigned update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); + ~KeeperAsynchronousMetrics() override; private: ContextPtr context; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index aaea0388239..ea5a3f19638 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -157,7 +157,7 @@ namespace DB M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ - M(UInt64, memory_worker_period_ms, 100, "Period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage.", 0) \ + M(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 872a9f864df..079029695c9 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -55,9 +55,11 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( ContextPtr global_context_, unsigned update_period_seconds, unsigned heavy_metrics_update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) : WithContext(global_context_) - , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index e3c83dc748e..5fab419a32b 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -14,7 +14,10 @@ public: ContextPtr global_context_, unsigned update_period_seconds, unsigned heavy_metrics_update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); + ~ServerAsynchronousMetrics() override; private: From 5a1b96ac8453f73de5e891e3a9f235fd96270b50 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 10:52:14 +0200 Subject: [PATCH 10/40] Style fix --- src/Common/CgroupsMemoryUsageObserver.cpp | 6 ------ src/Common/MemoryWorker.cpp | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index afeac1808b2..16d5d1cccde 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -19,12 +19,6 @@ using namespace DB; namespace DB { -namespace ErrorCodes -{ -extern const int FILE_DOESNT_EXIST; -extern const int INCORRECT_DATA; -} - CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) {} diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 42e797a80d6..ddc3fd783f4 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -30,7 +30,7 @@ namespace DB namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; - extern const int INCORRECT_DATA; + extern const int LOGICAL_ERROR; } #if defined(OS_LINUX) From 5b51a35e015336227dcc5b25445fefbc3da3059c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 11:31:34 +0200 Subject: [PATCH 11/40] Add unused variables --- src/Common/AsynchronousMetrics.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index eb6ede7a558..fedba4e55be 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -112,8 +112,8 @@ private: MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex); #endif - const bool update_jemalloc_epoch; - const bool update_rss; + [[maybe_unused]] const bool update_jemalloc_epoch; + [[maybe_unused]] const bool update_rss; #if defined(OS_LINUX) std::optional meminfo TSA_GUARDED_BY(data_mutex); From ade79cfd7a8465bf6e332f3b2ca6143ba652251a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 11:50:56 +0200 Subject: [PATCH 12/40] More fixes --- src/Common/MemoryWorker.cpp | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index ddc3fd783f4..8169bc7d177 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -186,9 +186,6 @@ std::shared_ptr createCgroupsReader() } #endif -constexpr uint64_t cgroups_memory_usage_tick_ms{50}; -constexpr uint64_t jemalloc_memory_usage_tick_ms{100}; - std::string_view sourceToString(MemoryWorker::MemoryUsageSource source) { switch (source) @@ -213,6 +210,8 @@ MemoryWorker::MemoryWorker(uint64_t period_ms_) #if defined(OS_LINUX) try { + static constexpr uint64_t cgroups_memory_usage_tick_ms{50}; + cgroups_reader = createCgroupsReader(); source = MemoryUsageSource::Cgroups; if (period_ms == 0) @@ -227,6 +226,8 @@ MemoryWorker::MemoryWorker(uint64_t period_ms_) #endif #if USE_JEMALLOC + static constexpr uint64_t jemalloc_memory_usage_tick_ms{100}; + source = MemoryUsageSource::Jemalloc; if (period_ms == 0) period_ms = jemalloc_memory_usage_tick_ms; @@ -270,7 +271,11 @@ uint64_t MemoryWorker::getMemoryUsage() case MemoryUsageSource::Cgroups: return cgroups_reader->readMemoryUsage(); case MemoryUsageSource::Jemalloc: +#if USE_JEMALLOC return resident_mib.getValue(); +#else + return 0; +#endif case MemoryUsageSource::None: throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to fetch memory usage while no memory source can be used"); } @@ -279,7 +284,7 @@ uint64_t MemoryWorker::getMemoryUsage() void MemoryWorker::backgroundThread() { std::chrono::milliseconds chrono_period_ms{period_ms}; - bool first_run = true; + [[maybe_unused]] bool first_run = true; std::unique_lock lock(mutex); while (true) { @@ -289,8 +294,10 @@ void MemoryWorker::backgroundThread() Stopwatch total_watch; +#if USE_JEMALLOC if (source == MemoryUsageSource::Jemalloc) epoch_mib.setValue(0); +#endif Int64 resident = getMemoryUsage(); MemoryTracker::updateRSS(resident); From 04d80ec2763a0677f32e9f6190cf2bcda0ebf8b7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 16:13:07 +0200 Subject: [PATCH 13/40] Fix non-linux build --- src/Common/MemoryWorker.cpp | 4 ++-- src/Common/MemoryWorker.h | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 8169bc7d177..c576772d303 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -33,7 +33,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -#if defined(OS_LINUX) struct ICgroupsReader { virtual ~ICgroupsReader() = default; @@ -44,6 +43,7 @@ struct ICgroupsReader namespace { +#if defined(OS_LINUX) /// Format is /// kernel 5 /// rss 15 @@ -269,7 +269,7 @@ uint64_t MemoryWorker::getMemoryUsage() switch (source) { case MemoryUsageSource::Cgroups: - return cgroups_reader->readMemoryUsage(); + return cgroups_reader != nullptr ? cgroups_reader->readMemoryUsage() : 0; case MemoryUsageSource::Jemalloc: #if USE_JEMALLOC return resident_mib.getValue(); diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 6fde93d63ad..b1b0495bf14 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -49,9 +49,7 @@ private: MemoryUsageSource source{MemoryUsageSource::None}; -#if defined(OS_LINUX) std::shared_ptr cgroups_reader; -#endif #if USE_JEMALLOC JemallocMibCache epoch_mib{"epoch"}; From ec5459a60d0be15e7bc100bd257e2b5c65baf594 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 09:15:34 +0200 Subject: [PATCH 14/40] Update allocated with resident if no jemalloc --- src/Common/MemoryWorker.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index c576772d303..75d0e7c32d8 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -310,15 +310,19 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } +#endif if (unlikely(first_run || total_memory_tracker.get() < 0)) { +#if USE_JEMALLOC if (source != MemoryUsageSource::Jemalloc) epoch_mib.setValue(0); MemoryTracker::updateAllocated(allocated_mib.getValue()); - } +#elif defined(OS_LINUX) + MemoryTracker::updateAllocated(resident); #endif + } ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); From e2e4c8ee0f8fcf1d6ef4d566c2fa4f9ee2123a56 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 10:21:09 +0200 Subject: [PATCH 15/40] Better --- src/Common/MemoryWorker.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 84ccffb8e90..e148f7f8f49 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -65,11 +65,25 @@ Metrics readAllMetricsFromStatFile(ReadBufferFromFile & buf) return metrics; } -uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) +uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, std::string_view key) { - const auto all_metrics = readAllMetricsFromStatFile(buf); - if (const auto it = all_metrics.find(key); it != all_metrics.end()) - return it->second; + while (!buf.eof()) + { + std::string current_key; + readStringUntilWhitespace(current_key, buf); + if (current_key != key) + { + std::string dummy; + readStringUntilNewlineInto(dummy, buf); + buf.ignore(); + continue; + } + + assertChar(' ', buf); + uint64_t value = 0; + readIntText(value, buf); + return value; + } LOG_ERROR(getLogger("CgroupsReader"), "Cannot find '{}' in '{}'", key, buf.getFileName()); return 0; } From f449c2fea0487abbe262c10fc5af9a99df1bc822 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Jul 2024 08:45:08 +0200 Subject: [PATCH 16/40] Fix --- src/Common/MemoryTracker.h | 5 +++++ src/Common/MemoryWorker.cpp | 8 +++----- src/Coordination/KeeperDispatcher.cpp | 3 ++- src/Coordination/KeeperServer.cpp | 2 +- 4 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index d2db8489f19..f15465a20c1 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -120,6 +120,11 @@ public: return amount.load(std::memory_order_relaxed); } + Int64 getRSS() const + { + return rss.load(std::memory_order_relaxed); + } + // Merges and mutations may pass memory ownership to other threads thus in the end of execution // MemoryTracker for background task may have a non-zero counter. // This method is intended to fix the counter inside of background_memory_tracker. diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index e148f7f8f49..1b869ed9d6b 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -312,7 +312,7 @@ uint64_t MemoryWorker::getMemoryUsage() void MemoryWorker::backgroundThread() { std::chrono::milliseconds chrono_period_ms{period_ms}; - bool first_run = true; + [[maybe_unused]] bool first_run = true; std::unique_lock lock(mutex); while (true) { @@ -340,17 +340,15 @@ void MemoryWorker::backgroundThread() } #endif +#if USE_JEMALLOC if (unlikely(first_run || total_memory_tracker.get() < 0)) { -#if USE_JEMALLOC if (source != MemoryUsageSource::Jemalloc) epoch_mib.setValue(0); MemoryTracker::updateAllocated(allocated_mib.getValue()); -#elif defined(OS_LINUX) - MemoryTracker::updateAllocated(resident); -#endif } +#endif ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 4c2ccb8db64..893bb8e6082 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -150,10 +150,11 @@ void KeeperDispatcher::requestThread() { LOG_WARNING( log, - "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type " + "Processing requests refused because of max_memory_usage_soft_limit {}, the total allocated memory is {}, RSS is {}, request type " "is {}", ReadableSize(mem_soft_limit), ReadableSize(total_memory_tracker.get()), + ReadableSize(total_memory_tracker.getRSS()), request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index d40e5ef2e50..ad9e8d32caa 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -599,7 +599,7 @@ bool KeeperServer::isLeaderAlive() const bool KeeperServer::isExceedingMemorySoftLimit() const { Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit(); - return mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit; + return mem_soft_limit > 0 && std::max(total_memory_tracker.get(), total_memory_tracker.getRSS()) >= mem_soft_limit; } /// TODO test whether taking failed peer in count From 7e444136bbad7e80f3a1905bbea7fa4c7e9a8337 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Sep 2024 18:27:24 +0200 Subject: [PATCH 17/40] Use QueryPlan for horizontal part of merge --- src/Interpreters/MutationsInterpreter.cpp | 13 +- src/Storages/MergeTree/MergeTask.cpp | 412 ++++++++++++------ .../MergeTree/MergeTreeSequentialSource.cpp | 51 ++- .../MergeTree/MergeTreeSequentialSource.h | 3 + 4 files changed, 334 insertions(+), 145 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0b93b5989b1..a8d45caeeaf 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1217,9 +1217,16 @@ void MutationsInterpreter::Source::read( createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, - plan, *data, storage_snapshot, - part, required_columns, - apply_deleted_mask_, std::move(filter), context_, + plan, + *data, storage_snapshot, + part, + required_columns, + nullptr, + apply_deleted_mask_, + std::move(filter), + false, + false, + context_, getLogger("MutationsInterpreter")); } else diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index fa86bb31629..3bee2ecb0d9 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -38,6 +38,11 @@ #include #include #include +#include +#include +#include +#include +#include #include #include #include @@ -1206,12 +1211,204 @@ bool MergeTask::execute() } +/// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream +class ApplyMergeStep : public ITransformingStep /// TODO: is this transformation step? +{ +public: + ApplyMergeStep( + const DataStream & input_stream_, + const SortDescription & sort_description_, + const Names partition_key_columns_, + const MergeTreeData::MergingParams & merging_params_, + WriteBuffer * rows_sources_write_buf_, + UInt64 merge_block_size_rows_, + UInt64 merge_block_size_bytes_, + bool blocks_are_granules_size_, + bool cleanup_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + , sort_description(sort_description_) + , partition_key_columns(partition_key_columns_) + , merging_params(merging_params_) + , rows_sources_write_buf(rows_sources_write_buf_) + , merge_block_size_rows(merge_block_size_rows_) + , merge_block_size_bytes(merge_block_size_bytes_) + , blocks_are_granules_size(blocks_are_granules_size_) + , cleanup(cleanup_) + {} + + String getName() const override { return "ApplyMergePolicy"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override + { + /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. + /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, + /// that is going in insertion order. + ProcessorPtr merged_transform; + +// /// There is no sense to have the block size bigger than one granule for merge operations. +// const UInt64 merge_block_size_rows = data_settings->merge_max_block_size; +// const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes; + + const auto &header = pipeline.getHeader(); + const auto input_streams_count = pipeline.getNumStreams(); + + switch (merging_params.mode) + { + case MergeTreeData::MergingParams::Ordinary: + merged_transform = std::make_shared( + header, + input_streams_count, + sort_description, + merge_block_size_rows, + merge_block_size_bytes, + SortingQueueStrategy::Default, + /* limit_= */0, + /* always_read_till_end_= */false, + rows_sources_write_buf, + blocks_are_granules_size); + break; + + case MergeTreeData::MergingParams::Collapsing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.sign_column, false, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + break; + + case MergeTreeData::MergingParams::Summing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes); + break; + + case MergeTreeData::MergingParams::Aggregating: + merged_transform = std::make_shared(header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes); + break; + + case MergeTreeData::MergingParams::Replacing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.is_deleted_column, merging_params.version_column, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size, + cleanup); + break; + + case MergeTreeData::MergingParams::Graphite: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes, + merging_params.graphite_params, time_of_merge); + break; + + case MergeTreeData::MergingParams::VersionedCollapsing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.sign_column, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + break; + } + + pipeline.addTransform(std::move(merged_transform)); + +#ifndef NDEBUG + if (!sort_description.empty()) + { + pipeline.addSimpleTransform([&](const Block & header_) + { + auto transform = std::make_shared(header_, sort_description); + return transform; + }); + } +#endif + } + + void updateOutputStream() override + { + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + output_stream->sort_description = sort_description; + + /// TODO: is this correct? +// if (partition_key_columns.empty()) + output_stream->sort_scope = DataStream::SortScope::Global; +// else +// output_stream->sort_scope = DataStream::SortScope::Stream; + } + +private: + SortDescription sort_description; + Names partition_key_columns; + MergeTreeData::MergingParams merging_params{}; + WriteBuffer * rows_sources_write_buf; + const UInt64 merge_block_size_rows; + const UInt64 merge_block_size_bytes; + bool blocks_are_granules_size; + bool cleanup{false}; + time_t time_of_merge{0}; +}; + + +class MaterializingStep : public ITransformingStep /// TODO: is this transformation step? +{ +public: + explicit MaterializingStep( + const DataStream & input_stream_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + {} + + String getName() const override { return "Materializing"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + pipeline.addTransform(std::make_shared(input_streams.front().header)); + } + + void updateOutputStream() override + { + /// TODO: can this be simplified? + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + output_stream->sort_description = input_streams.front().sort_description; + } +}; + + +class TTLStep : public ITransformingStep +{ +public: + TTLStep( + const DataStream & input_stream_, + const ContextPtr & context_, + const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const MergeTreeData::MutableDataPartPtr & data_part_, + time_t current_time, + bool force_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + { + transform = std::make_shared(context_, input_stream_.header, storage_, metadata_snapshot_, data_part_, current_time, force_); + subqueries_for_sets = transform->getSubqueries(); + } + + String getName() const override { return "Materializing"; } + + PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + pipeline.addTransform(transform); + } + + void updateOutputStream() override + { + // TODO: implement? + } + +private: + std::shared_ptr transform; + PreparedSets::Subqueries subqueries_for_sets; +}; + + void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() { /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ - Pipes pipes; + global_ctx->watch_prev_elapsed = 0; /// We count total amount of bytes in parts @@ -1238,143 +1435,92 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); + auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone(); + + /// Read from all parts + std::vector plans; for (const auto & part : global_ctx->future_part->parts) { - Pipe pipe = createMergeTreeSequentialSource( + /// TODO: this is just for debugging purposes, remove it later + if (part->getMarksCount() == 0) + LOG_DEBUG(ctx->log, "Part {} is empty", part->name); + + auto plan_for_part = std::make_unique(); + createReadFromPartStep( MergeTreeSequentialSourceType::Merge, + *plan_for_part, *global_ctx->data, global_ctx->storage_snapshot, part, global_ctx->merging_columns.getNames(), - /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, /*apply_deleted_mask=*/ true, + /*filter=*/ std::nullopt, ctx->read_with_direct_io, - /*prefetch=*/ false); + /*prefetch=*/ false, + global_ctx->context, + ctx->log); if (global_ctx->metadata_snapshot->hasSortingKey()) { - pipe.addSimpleTransform([this](const Block & header) - { - return std::make_shared(header, global_ctx->metadata_snapshot->getSortingKey().expression); - }); + /// Calculate sorting key expressions so that they are available for merge sorting. + auto calculate_sorting_key_expression_step = std::make_unique( + plan_for_part->getCurrentDataStream(), + sorting_key_expression_dag.clone()); /// TODO: can we avoid cloning here? + plan_for_part->addStep(std::move(calculate_sorting_key_expression_step)); } - pipes.emplace_back(std::move(pipe)); + plans.emplace_back(std::move(plan_for_part)); } + QueryPlan merge_parts_query_plan; - Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); - SortDescription sort_description; - sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description; - sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef().min_count_to_compile_sort_description; - - size_t sort_columns_size = sort_columns.size(); - sort_description.reserve(sort_columns_size); - - Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names; - - Block header = pipes.at(0).getHeader(); - for (size_t i = 0; i < sort_columns_size; ++i) - sort_description.emplace_back(sort_columns[i], 1, 1); - -#ifndef NDEBUG - if (!sort_description.empty()) + /// Union of all parts streams { - for (size_t i = 0; i < pipes.size(); ++i) - { - auto & pipe = pipes[i]; - pipe.addSimpleTransform([&](const Block & header_) - { - auto transform = std::make_shared(header_, sort_description); - transform->setDescription(global_ctx->future_part->parts[i]->name); - return transform; - }); - } + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); } -#endif - /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. - /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, - /// that is going in insertion order. - ProcessorPtr merged_transform; - - /// If merge is vertical we cannot calculate it - ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); - - /// There is no sense to have the block size bigger than one granule for merge operations. - const UInt64 merge_block_size_rows = data_settings->merge_max_block_size; - const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes; - - switch (ctx->merging_params.mode) + /// Merge { - case MergeTreeData::MergingParams::Ordinary: - merged_transform = std::make_shared( - header, - pipes.size(), - sort_description, - merge_block_size_rows, - merge_block_size_bytes, - SortingQueueStrategy::Default, - /* limit_= */0, - /* always_read_till_end_= */false, - ctx->rows_sources_write_buf.get(), - ctx->blocks_are_granules_size); - break; + Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); + SortDescription sort_description; + sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description; + sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef().min_count_to_compile_sort_description; - case MergeTreeData::MergingParams::Collapsing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.sign_column, false, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size); - break; + size_t sort_columns_size = sort_columns.size(); + sort_description.reserve(sort_columns_size); - case MergeTreeData::MergingParams::Summing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes); - break; + Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names; - case MergeTreeData::MergingParams::Aggregating: - merged_transform = std::make_shared(header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes); - break; + for (size_t i = 0; i < sort_columns_size; ++i) + sort_description.emplace_back(sort_columns[i], 1, 1); - case MergeTreeData::MergingParams::Replacing: - if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); + /// If merge is vertical we cannot calculate it + ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.is_deleted_column, ctx->merging_params.version_column, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size, - global_ctx->cleanup); - break; + if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); - case MergeTreeData::MergingParams::Graphite: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes, - ctx->merging_params.graphite_params, global_ctx->time_of_merge); - break; - - case MergeTreeData::MergingParams::VersionedCollapsing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.sign_column, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size); - break; + auto merge_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + sort_description, + partition_key_columns, + ctx->merging_params, + ctx->rows_sources_write_buf.get(), + data_settings->merge_max_block_size, + data_settings->merge_max_block_size_bytes, + ctx->blocks_are_granules_size, + global_ctx->cleanup); + merge_step->setStepDescription("Merge sorted parts"); + merge_parts_query_plan.addStep(std::move(merge_step)); } - auto builder = std::make_unique(); - builder->init(Pipe::unitePipes(std::move(pipes))); - builder->addTransform(std::move(merged_transform)); - -#ifndef NDEBUG - if (!sort_description.empty()) - { - builder->addSimpleTransform([&](const Block & header_) - { - auto transform = std::make_shared(header_, sort_description); - return transform; - }); - } -#endif - if (global_ctx->deduplicate) { const auto & virtuals = *global_ctx->data->getVirtualsPtr(); @@ -1383,44 +1529,56 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() /// If deduplicate_by_columns is empty, add all columns except virtuals. if (global_ctx->deduplicate_by_columns.empty()) { - for (const auto & column : global_ctx->merging_columns) + for (const auto & column_name : global_ctx->merging_columns.getNames()) { - if (virtuals.tryGet(column.name, VirtualsKind::Persistent)) + if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) continue; - global_ctx->deduplicate_by_columns.emplace_back(column.name); + global_ctx->deduplicate_by_columns.emplace_back(column_name); } } - if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns)) - builder->addTransform(std::make_shared( - builder->getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); - else - builder->addTransform(std::make_shared( - builder->getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); + auto deduplication_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + SizeLimits(), 0 /*limit_hint*/, + global_ctx->deduplicate_by_columns, + false, + true /*TODO: ??*/); + deduplication_step->setStepDescription("Deduplication step"); + merge_parts_query_plan.addStep(std::move(deduplication_step)); } PreparedSets::Subqueries subqueries; + /// TTL step if (ctx->need_remove_expired_values) { - auto transform = std::make_shared(global_ctx->context, builder->getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); - subqueries = transform->getSubqueries(); - builder->addTransform(std::move(transform)); + auto ttl_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), global_ctx->context, *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); + subqueries = ttl_step->getSubqueries(); + ttl_step->setStepDescription("TTL step"); + merge_parts_query_plan.addStep(std::move(ttl_step)); } + /// Secondary indices expressions if (!global_ctx->merging_skip_indexes.empty()) { - builder->addTransform(std::make_shared( - builder->getHeader(), - global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), - global_ctx->data->getContext()))); - - builder->addTransform(std::make_shared(builder->getHeader())); + auto indices_expression_dag = global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + auto calculate_indices_expression_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + std::move(indices_expression_dag)); + merge_parts_query_plan.addStep(std::move(calculate_indices_expression_step)); + /// TODO: what is the purpose of MaterializingTransform in the original code? + merge_parts_query_plan.addStep(std::make_unique(merge_parts_query_plan.getCurrentDataStream())); } if (!subqueries.empty()) - builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context); + addCreatingSetsStep(merge_parts_query_plan, std::move(subqueries), global_ctx->context); + + auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto builder = merge_parts_query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(global_ctx->context), + pipelineSettings); global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); /// Dereference unique_ptr and pass horizontal_stage_progress by reference diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 39aa191a3d2..444a59b5590 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -347,8 +347,11 @@ public: const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, + std::shared_ptr> filtered_rows_count_, bool apply_deleted_mask_, std::optional filter_, + bool read_with_direct_io_, + bool prefetch_, ContextPtr context_, LoggerPtr log_) : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) @@ -357,8 +360,11 @@ public: , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) , columns_to_read(std::move(columns_to_read_)) + , filtered_rows_count(std::move(filtered_rows_count_)) , apply_deleted_mask(apply_deleted_mask_) , filter(std::move(filter_)) + , read_with_direct_io(read_with_direct_io_) + , prefetch(prefetch_) , context(std::move(context_)) , log(log_) { @@ -401,24 +407,27 @@ public: data_part, columns_to_read, std::move(mark_ranges), - /*filtered_rows_count=*/ nullptr, + filtered_rows_count, apply_deleted_mask, - /*read_with_direct_io=*/ false, - /*prefetch=*/ false); + read_with_direct_io, + prefetch); pipeline.init(Pipe(std::move(source))); } private: - MergeTreeSequentialSourceType type; + const MergeTreeSequentialSourceType type; const MergeTreeData & storage; - StorageSnapshotPtr storage_snapshot; - MergeTreeData::DataPartPtr data_part; - Names columns_to_read; - bool apply_deleted_mask; - std::optional filter; - ContextPtr context; - LoggerPtr log; + const StorageSnapshotPtr storage_snapshot; + const MergeTreeData::DataPartPtr data_part; + const Names columns_to_read; + const std::shared_ptr> filtered_rows_count; + const bool apply_deleted_mask; + const std::optional filter; + const bool read_with_direct_io; + const bool prefetch; + const ContextPtr context; + const LoggerPtr log; }; void createReadFromPartStep( @@ -428,15 +437,27 @@ void createReadFromPartStep( const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, Names columns_to_read, + std::shared_ptr> filtered_rows_count, bool apply_deleted_mask, std::optional filter, + bool read_with_direct_io, + bool prefetch, ContextPtr context, LoggerPtr log) { - auto reading = std::make_unique(type, - storage, storage_snapshot, std::move(data_part), - std::move(columns_to_read), apply_deleted_mask, - std::move(filter), std::move(context), log); + auto reading = std::make_unique( + type, + storage, + storage_snapshot, + std::move(data_part), + std::move(columns_to_read), + filtered_rows_count, + apply_deleted_mask, + std::move(filter), + read_with_direct_io, + prefetch, + std::move(context), + log); plan.addStep(std::move(reading)); } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index 1b05512b9a3..543d1f60d10 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -37,8 +37,11 @@ void createReadFromPartStep( const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, Names columns_to_read, + std::shared_ptr> filtered_rows_count, bool apply_deleted_mask, std::optional filter, + bool read_with_direct_io, + bool prefetch, ContextPtr context, LoggerPtr log); From 13f4eb3fac6c2c0781351ee5db382383193b2af5 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Sep 2024 22:24:53 +0200 Subject: [PATCH 18/40] Fix for graphite merge mode --- src/Storages/MergeTree/MergeTask.cpp | 25 ++++++++++++------------- src/Storages/MergeTree/MergeTask.h | 2 +- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 3bee2ecb0d9..fb5bbc4729c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1224,7 +1224,8 @@ public: UInt64 merge_block_size_rows_, UInt64 merge_block_size_bytes_, bool blocks_are_granules_size_, - bool cleanup_) + bool cleanup_, + time_t time_of_merge_) : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? , sort_description(sort_description_) , partition_key_columns(partition_key_columns_) @@ -1234,6 +1235,7 @@ public: , merge_block_size_bytes(merge_block_size_bytes_) , blocks_are_granules_size(blocks_are_granules_size_) , cleanup(cleanup_) + , time_of_merge(time_of_merge_) {} String getName() const override { return "ApplyMergePolicy"; } @@ -1245,10 +1247,6 @@ public: /// that is going in insertion order. ProcessorPtr merged_transform; -// /// There is no sense to have the block size bigger than one granule for merge operations. -// const UInt64 merge_block_size_rows = data_settings->merge_max_block_size; -// const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes; - const auto &header = pipeline.getHeader(); const auto input_streams_count = pipeline.getNumStreams(); @@ -1330,15 +1328,15 @@ public: } private: - SortDescription sort_description; - Names partition_key_columns; - MergeTreeData::MergingParams merging_params{}; + const SortDescription sort_description; + const Names partition_key_columns; + const MergeTreeData::MergingParams merging_params{}; WriteBuffer * rows_sources_write_buf; const UInt64 merge_block_size_rows; const UInt64 merge_block_size_bytes; - bool blocks_are_granules_size; - bool cleanup{false}; - time_t time_of_merge{0}; + const bool blocks_are_granules_size; + const bool cleanup{false}; + const time_t time_of_merge{0}; }; @@ -1403,7 +1401,7 @@ private: }; -void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() +void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const { /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. @@ -1516,7 +1514,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() data_settings->merge_max_block_size, data_settings->merge_max_block_size_bytes, ctx->blocks_are_granules_size, - global_ctx->cleanup); + global_ctx->cleanup, + global_ctx->time_of_merge); merge_step->setStepDescription("Merge sorted parts"); merge_parts_query_plan.addStep(std::move(merge_step)); } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index c80995888d4..a5d7851932c 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -291,7 +291,7 @@ private: bool executeMergeProjections(); MergeAlgorithm chooseMergeAlgorithm() const; - void createMergedStream(); + void createMergedStream() const; void extractMergingAndGatheringColumns() const; void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override From 48cacd6f310c107c1dd0239a7639527adb054b69 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Sep 2024 22:36:42 +0200 Subject: [PATCH 19/40] Use query plan for column vertical merges --- src/Storages/MergeTree/MergeTask.cpp | 175 ++++++++++++++++++++------- src/Storages/MergeTree/MergeTask.h | 5 +- 2 files changed, 133 insertions(+), 47 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index fb5bbc4729c..75fd61ae4be 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -804,35 +804,106 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const bool all_parts_on_remote_disks = std::ranges::all_of(global_ctx->future_part->parts, [](const auto & part) { return part->isStoredOnRemoteDisk(); }); ctx->use_prefetch = all_parts_on_remote_disks && global_ctx->data->getSettings()->vertical_merge_remote_filesystem_prefetch; - if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) - ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name); +// if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) +// ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name); return false; } -Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String & column_name) const +QueryPlan MergeTask::VerticalMergeStage::createPlanForReadingOneColumn(const String & column_name) const { - Pipes pipes; - for (size_t part_num = 0; part_num < global_ctx->future_part->parts.size(); ++part_num) + /// Read from all parts + std::vector plans; + for (const auto & part : global_ctx->future_part->parts) { - Pipe pipe = createMergeTreeSequentialSource( + auto plan_for_part = std::make_unique(); + createReadFromPartStep( MergeTreeSequentialSourceType::Merge, + *plan_for_part, *global_ctx->data, global_ctx->storage_snapshot, - global_ctx->future_part->parts[part_num], + part, Names{column_name}, - /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, /*apply_deleted_mask=*/ true, + std::nullopt, ctx->read_with_direct_io, - ctx->use_prefetch); + ctx->use_prefetch, + global_ctx->context, + getLogger("VerticalMergeStage")); - pipes.emplace_back(std::move(pipe)); + plans.emplace_back(std::move(plan_for_part)); } - return Pipe::unitePipes(std::move(pipes)); + QueryPlan merge_parts_query_plan; + + /// Union of all parts streams + { + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); + } + + return merge_parts_query_plan; } +/// Gathers values from all parts for one column using rows sources temporary file +class ColumnGathererStep : public ITransformingStep +{ +public: + ColumnGathererStep( + const DataStream & input_stream_, + CompressedReadBufferFromFile * rows_sources_read_buf_, + UInt64 merge_block_size_rows_, + UInt64 merge_block_size_bytes_, + bool is_result_sparse_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + , rows_sources_read_buf(rows_sources_read_buf_) + , merge_block_size_rows(merge_block_size_rows_) + , merge_block_size_bytes(merge_block_size_bytes_) + , is_result_sparse(is_result_sparse_) + {} + + String getName() const override { return "ColumnGatherer"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override + { + const auto &header = pipeline.getHeader(); + const auto input_streams_count = pipeline.getNumStreams(); + + rows_sources_read_buf->seek(0, 0); + + auto transform = std::make_unique( + header, + input_streams_count, + *rows_sources_read_buf, + merge_block_size_rows, + merge_block_size_bytes, + is_result_sparse); + + pipeline.addTransform(std::move(transform)); + } + + void updateOutputStream() override + { + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + + /// TODO: is this correct? + output_stream->sort_scope = DataStream::SortScope::None; + } + +private: + MergeTreeData::MergingParams merging_params{}; + CompressedReadBufferFromFile * rows_sources_read_buf; + const UInt64 merge_block_size_rows; + const UInt64 merge_block_size_bytes; + const bool is_result_sparse; +}; + void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const { const auto & column_name = ctx->it_name_and_type->name; @@ -840,50 +911,64 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); - Pipe pipe; - if (ctx->prepared_pipe) - { - pipe = std::move(*ctx->prepared_pipe); +// Pipe pipe; +//// if (ctx->prepared_pipe) +//// { +//// pipe = std::move(*ctx->prepared_pipe); +//// +//// auto next_column_it = std::next(ctx->it_name_and_type); +//// if (next_column_it != global_ctx->gathering_columns.end()) +//// ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name); +//// } +//// else +// { +// pipe = createPipeForReadingOneColumn(column_name); +// } - auto next_column_it = std::next(ctx->it_name_and_type); - if (next_column_it != global_ctx->gathering_columns.end()) - ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name); - } - else + auto merge_column_query_plan = createPlanForReadingOneColumn(column_name); + + /// Add column gatherer step { - pipe = createPipeForReadingOneColumn(column_name); +// ctx->rows_sources_read_buf->seek(0, 0); + bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; + const auto data_settings = global_ctx->data->getSettings(); + auto merge_step = std::make_unique( + merge_column_query_plan.getCurrentDataStream(), + ctx->rows_sources_read_buf.get(), //global_ctx->rows_sources_temporary_file_name, + data_settings->merge_max_block_size, + data_settings->merge_max_block_size_bytes, + is_result_sparse); + merge_step->setStepDescription("Gather column"); + merge_column_query_plan.addStep(std::move(merge_step)); } - ctx->rows_sources_read_buf->seek(0, 0); - bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; - - const auto data_settings = global_ctx->data->getSettings(); - auto transform = std::make_unique( - pipe.getHeader(), - pipe.numOutputPorts(), - *ctx->rows_sources_read_buf, - data_settings->merge_max_block_size, - data_settings->merge_max_block_size_bytes, - is_result_sparse); - - pipe.addTransform(std::move(transform)); - + /// Add expression step for indexes MergeTreeIndices indexes_to_recalc; - auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); - - if (indexes_it != global_ctx->skip_indexes_by_column.end()) + IndicesDescription indexes_to_recalc_description; { - indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); + auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); - pipe.addTransform(std::make_shared( - pipe.getHeader(), - indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), - global_ctx->data->getContext()))); + if (indexes_it != global_ctx->skip_indexes_by_column.end()) + { + indexes_to_recalc_description = indexes_it->second; + indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); - pipe.addTransform(std::make_shared(pipe.getHeader())); + auto indices_expression_dag = indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + auto calculate_indices_expression_step = std::make_unique( + merge_column_query_plan.getCurrentDataStream(), + std::move(indices_expression_dag)); + merge_column_query_plan.addStep(std::move(calculate_indices_expression_step)); + } } - ctx->column_parts_pipeline = QueryPipeline(std::move(pipe)); + { + auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto builder = merge_column_query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(global_ctx->context), + pipelineSettings); + + ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + } /// Dereference unique_ptr ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback( diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index a5d7851932c..b36f5f832d9 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -333,7 +333,8 @@ private: Float64 progress_before = 0; std::unique_ptr column_to{nullptr}; - std::optional prepared_pipe; +// TODO: is this really needed for prefetch? +// std::optional prepared_pipe; size_t max_delayed_streams = 0; bool use_prefetch = false; std::list> delayed_streams; @@ -378,7 +379,7 @@ private: bool executeVerticalMergeForOneColumn() const; void finalizeVerticalMergeForOneColumn() const; - Pipe createPipeForReadingOneColumn(const String & column_name) const; + QueryPlan createPlanForReadingOneColumn(const String & column_name) const; VerticalMergeRuntimeContextPtr ctx; GlobalRuntimeContextPtr global_ctx; From d28cba981ccd6a58939854a0204d654c6075337d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 08:59:01 +0200 Subject: [PATCH 20/40] Fix clang_tidy --- src/Storages/MergeTree/MergeTask.cpp | 6 +++--- src/Storages/MergeTree/MergeTask.h | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 75fd61ae4be..cafc11fc34d 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -195,7 +195,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu } } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const { ProfileEvents::increment(ProfileEvents::Merge); @@ -657,7 +657,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPart } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // NOLINT +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() const { /// In case if there are no projections we didn't construct a task if (!ctx->merge_projection_parts_task_ptr) @@ -676,7 +676,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // N return true; } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() const { Stopwatch watch(CLOCK_MONOTONIC_COARSE); UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index b36f5f832d9..a30ab4712d5 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -268,12 +268,12 @@ private: { bool execute() override; - bool prepare(); - bool executeImpl(); + bool prepare() const; + bool executeImpl() const; void finalize() const; /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable - using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; + using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; const ExecuteAndFinalizeHorizontalPartSubtasks subtasks { @@ -288,7 +288,7 @@ private: void calculateProjections(const Block & block) const; void finalizeProjections() const; void constructTaskForProjectionPartsMerge() const; - bool executeMergeProjections(); + bool executeMergeProjections() const; MergeAlgorithm chooseMergeAlgorithm() const; void createMergedStream() const; From 6a6935cb84a31493def51cf5d65954bec75f587e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 13:09:18 +0200 Subject: [PATCH 21/40] Cleanup --- src/Storages/MergeTree/MergeTask.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index cafc11fc34d..6f5b8301d4a 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1466,7 +1466,7 @@ public: subqueries_for_sets = transform->getSubqueries(); } - String getName() const override { return "Materializing"; } + String getName() const override { return "TTL"; } PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } @@ -1524,9 +1524,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const std::vector plans; for (const auto & part : global_ctx->future_part->parts) { - /// TODO: this is just for debugging purposes, remove it later if (part->getMarksCount() == 0) - LOG_DEBUG(ctx->log, "Part {} is empty", part->name); + LOG_TRACE(ctx->log, "Part {} is empty", part->name); auto plan_for_part = std::make_unique(); createReadFromPartStep( @@ -1613,12 +1612,12 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const /// If deduplicate_by_columns is empty, add all columns except virtuals. if (global_ctx->deduplicate_by_columns.empty()) { - for (const auto & column_name : global_ctx->merging_columns.getNames()) + for (const auto & column : global_ctx->merging_columns) { - if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) + if (virtuals.tryGet(column.name, VirtualsKind::Persistent)) continue; - global_ctx->deduplicate_by_columns.emplace_back(column_name); + global_ctx->deduplicate_by_columns.emplace_back(column.name); } } From a1cec53b7c2a6508277280bd8c36f90dfe661560 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 14:54:05 +0200 Subject: [PATCH 22/40] Fix updateOutputStream and Traits --- src/Storages/MergeTree/MergeTask.cpp | 106 ++++++++++++++++++++------- 1 file changed, 78 insertions(+), 28 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 6f5b8301d4a..1bf1573fc1f 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -861,7 +861,7 @@ public: UInt64 merge_block_size_rows_, UInt64 merge_block_size_bytes_, bool is_result_sparse_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , rows_sources_read_buf(rows_sources_read_buf_) , merge_block_size_rows(merge_block_size_rows_) , merge_block_size_bytes(merge_block_size_bytes_) @@ -891,12 +891,24 @@ public: void updateOutputStream() override { output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - - /// TODO: is this correct? - output_stream->sort_scope = DataStream::SortScope::None; } private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + MergeTreeData::MergingParams merging_params{}; CompressedReadBufferFromFile * rows_sources_read_buf; const UInt64 merge_block_size_rows; @@ -962,10 +974,9 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const } { - auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); - auto builder = merge_column_query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(global_ctx->context), - pipelineSettings); + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); } @@ -1297,7 +1308,7 @@ bool MergeTask::execute() /// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream -class ApplyMergeStep : public ITransformingStep /// TODO: is this transformation step? +class ApplyMergeStep : public ITransformingStep { public: ApplyMergeStep( @@ -1311,7 +1322,7 @@ public: bool blocks_are_granules_size_, bool cleanup_, time_t time_of_merge_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , sort_description(sort_description_) , partition_key_columns(partition_key_columns_) , merging_params(merging_params_) @@ -1403,16 +1414,24 @@ public: void updateOutputStream() override { output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - output_stream->sort_description = sort_description; - - /// TODO: is this correct? -// if (partition_key_columns.empty()) - output_stream->sort_scope = DataStream::SortScope::Global; -// else -// output_stream->sort_scope = DataStream::SortScope::Stream; } private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + const SortDescription sort_description; const Names partition_key_columns; const MergeTreeData::MergingParams merging_params{}; @@ -1425,12 +1444,12 @@ private: }; -class MaterializingStep : public ITransformingStep /// TODO: is this transformation step? +class MaterializingStep : public ITransformingStep { public: explicit MaterializingStep( const DataStream & input_stream_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) {} String getName() const override { return "Materializing"; } @@ -1442,9 +1461,23 @@ public: void updateOutputStream() override { - /// TODO: can this be simplified? output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - output_stream->sort_description = input_streams.front().sort_description; + } + +private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = true, + } + }; } }; @@ -1460,7 +1493,7 @@ public: const MergeTreeData::MutableDataPartPtr & data_part_, time_t current_time, bool force_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) { transform = std::make_shared(context_, input_stream_.header, storage_, metadata_snapshot_, data_part_, current_time, force_); subqueries_for_sets = transform->getSubqueries(); @@ -1477,10 +1510,25 @@ public: void updateOutputStream() override { - // TODO: implement? + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); } private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + std::shared_ptr transform; PreparedSets::Subqueries subqueries_for_sets; }; @@ -1658,12 +1706,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const if (!subqueries.empty()) addCreatingSetsStep(merge_parts_query_plan, std::move(subqueries), global_ctx->context); - auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); - auto builder = merge_parts_query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(global_ctx->context), - pipelineSettings); + { + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_parts_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); + + global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + } - global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); /// Dereference unique_ptr and pass horizontal_stage_progress by reference global_ctx->merged_pipeline.setProgressCallback(MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress)); /// Is calculated inside MergeProgressCallback. From 8361724539408d95f9757e00047919d70ea50bbd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 17:02:25 +0200 Subject: [PATCH 23/40] Build pipeline for next column for prefetching --- src/Storages/MergeTree/MergeTask.cpp | 140 +++++++++++++-------------- src/Storages/MergeTree/MergeTask.h | 14 ++- 2 files changed, 81 insertions(+), 73 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 1bf1573fc1f..a4104672de7 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -804,53 +804,12 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const bool all_parts_on_remote_disks = std::ranges::all_of(global_ctx->future_part->parts, [](const auto & part) { return part->isStoredOnRemoteDisk(); }); ctx->use_prefetch = all_parts_on_remote_disks && global_ctx->data->getSettings()->vertical_merge_remote_filesystem_prefetch; -// if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) -// ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name); + if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) + ctx->prepared_pipeline = createPipelineForReadingOneColumn(ctx->it_name_and_type->name); return false; } -QueryPlan MergeTask::VerticalMergeStage::createPlanForReadingOneColumn(const String & column_name) const -{ - /// Read from all parts - std::vector plans; - for (const auto & part : global_ctx->future_part->parts) - { - auto plan_for_part = std::make_unique(); - createReadFromPartStep( - MergeTreeSequentialSourceType::Merge, - *plan_for_part, - *global_ctx->data, - global_ctx->storage_snapshot, - part, - Names{column_name}, - global_ctx->input_rows_filtered, - /*apply_deleted_mask=*/ true, - std::nullopt, - ctx->read_with_direct_io, - ctx->use_prefetch, - global_ctx->context, - getLogger("VerticalMergeStage")); - - plans.emplace_back(std::move(plan_for_part)); - } - - QueryPlan merge_parts_query_plan; - - /// Union of all parts streams - { - DataStreams input_streams; - input_streams.reserve(plans.size()); - for (auto & plan : plans) - input_streams.emplace_back(plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams)); - merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); - } - - return merge_parts_query_plan; -} - /// Gathers values from all parts for one column using rows sources temporary file class ColumnGathererStep : public ITransformingStep { @@ -916,32 +875,46 @@ private: const bool is_result_sparse; }; -void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const +MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::VerticalMergeStage::createPipelineForReadingOneColumn(const String & column_name) const { - const auto & column_name = ctx->it_name_and_type->name; + /// Read from all parts + std::vector plans; + for (const auto & part : global_ctx->future_part->parts) + { + auto plan_for_part = std::make_unique(); + createReadFromPartStep( + MergeTreeSequentialSourceType::Merge, + *plan_for_part, + *global_ctx->data, + global_ctx->storage_snapshot, + part, + Names{column_name}, + global_ctx->input_rows_filtered, + /*apply_deleted_mask=*/ true, + std::nullopt, + ctx->read_with_direct_io, + ctx->use_prefetch, + global_ctx->context, + getLogger("VerticalMergeStage")); - ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); - global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); + plans.emplace_back(std::move(plan_for_part)); + } -// Pipe pipe; -//// if (ctx->prepared_pipe) -//// { -//// pipe = std::move(*ctx->prepared_pipe); -//// -//// auto next_column_it = std::next(ctx->it_name_and_type); -//// if (next_column_it != global_ctx->gathering_columns.end()) -//// ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name); -//// } -//// else -// { -// pipe = createPipeForReadingOneColumn(column_name); -// } + QueryPlan merge_column_query_plan; - auto merge_column_query_plan = createPlanForReadingOneColumn(column_name); + /// Union of all parts streams + { + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_column_query_plan.unitePlans(std::move(union_step), std::move(plans)); + } /// Add column gatherer step { -// ctx->rows_sources_read_buf->seek(0, 0); bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; const auto data_settings = global_ctx->data->getSettings(); auto merge_step = std::make_unique( @@ -973,13 +946,36 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const } } - { - auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); - auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); - auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); - ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + return {QueryPipelineBuilder::getPipeline(std::move(*builder)), std::move(indexes_to_recalc)}; +} + +void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const +{ + const auto & column_name = ctx->it_name_and_type->name; + + ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); + global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); + + VerticalMergeRuntimeContext::PreparedColumnPipeline column_pipepline; + if (ctx->prepared_pipeline) + { + column_pipepline = std::move(*ctx->prepared_pipeline); + + /// Prepare next column pipeline to initiate prefetching + auto next_column_it = std::next(ctx->it_name_and_type); + if (next_column_it != global_ctx->gathering_columns.end()) + ctx->prepared_pipeline = createPipelineForReadingOneColumn(next_column_it->name); } + else + { + column_pipepline = createPipelineForReadingOneColumn(column_name); + } + + ctx->column_parts_pipeline = std::move(column_pipepline.pipeline); /// Dereference unique_ptr ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback( @@ -997,12 +993,16 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const global_ctx->metadata_snapshot, columns_list, ctx->compression_codec, - indexes_to_recalc, + column_pipepline.indexes_to_recalc, getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, global_ctx->to->getIndexGranularity()); ctx->column_elems_written = 0; + + /// rows_sources_read_buf is reused for each column so we need to rewind it explicitly each time + /// This sharing also prevents from from running multiple merge of individual columns in parallel. + ctx->rows_sources_read_buf->seek(0, 0); } @@ -1673,8 +1673,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const merge_parts_query_plan.getCurrentDataStream(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns, - false, - true /*TODO: ??*/); + false /*pre_distinct*/, + true /*optimize_distinct_in_order TODO: looks like it shoud be enabled*/); deduplication_step->setStepDescription("Deduplication step"); merge_parts_query_plan.addStep(std::move(deduplication_step)); } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index a30ab4712d5..bbe53c34c7e 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -333,8 +333,16 @@ private: Float64 progress_before = 0; std::unique_ptr column_to{nullptr}; -// TODO: is this really needed for prefetch? -// std::optional prepared_pipe; + + /// Used for prefetching. Right before starting merge of a column we create a pipeline for the next column + /// and it initiates prefetching of the first range of that column. + struct PreparedColumnPipeline + { + QueryPipeline pipeline; + MergeTreeIndices indexes_to_recalc; + }; + + std::optional prepared_pipeline; size_t max_delayed_streams = 0; bool use_prefetch = false; std::list> delayed_streams; @@ -379,7 +387,7 @@ private: bool executeVerticalMergeForOneColumn() const; void finalizeVerticalMergeForOneColumn() const; - QueryPlan createPlanForReadingOneColumn(const String & column_name) const; + VerticalMergeRuntimeContext::PreparedColumnPipeline createPipelineForReadingOneColumn(const String & column_name) const; VerticalMergeRuntimeContextPtr ctx; GlobalRuntimeContextPtr global_ctx; From 472e6eb856e338332fbebb2519066f093c18a15f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 17:16:43 +0200 Subject: [PATCH 24/40] typo --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index a4104672de7..576ea341877 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1674,7 +1674,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns, false /*pre_distinct*/, - true /*optimize_distinct_in_order TODO: looks like it shoud be enabled*/); + true /*optimize_distinct_in_order TODO: looks like it should be enabled*/); deduplication_step->setStepDescription("Deduplication step"); merge_parts_query_plan.addStep(std::move(deduplication_step)); } From 20eaecc4f39adf73ac402c88d4a54d70f859453c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 5 Sep 2024 13:50:26 +0200 Subject: [PATCH 25/40] Fix build --- src/Storages/MergeTree/MergeTask.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e6d7b4656c9..398a9472456 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1585,8 +1585,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const std::vector plans; for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) { - if (part->getMarksCount() == 0) - LOG_TRACE(ctx->log, "Part {} is empty", part->name); + if (global_ctx->future_part->parts[i]->getMarksCount() == 0) + LOG_TRACE(ctx->log, "Part {} is empty", global_ctx->future_part->parts[i]->name); auto plan_for_part = std::make_unique(); createReadFromPartStep( From 1bcc4ba823805bed282133fb7035b73598641fc6 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 9 Sep 2024 15:30:19 +0200 Subject: [PATCH 26/40] Renamed ApplyMergeStep into MergePartsStep --- src/Storages/MergeTree/MergeTask.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 398a9472456..3ca909a2d09 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1321,10 +1321,10 @@ bool MergeTask::execute() /// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream -class ApplyMergeStep : public ITransformingStep +class MergePartsStep : public ITransformingStep { public: - ApplyMergeStep( + MergePartsStep( const DataStream & input_stream_, const SortDescription & sort_description_, const Names partition_key_columns_, @@ -1347,7 +1347,7 @@ public: , time_of_merge(time_of_merge_) {} - String getName() const override { return "ApplyMergePolicy"; } + String getName() const override { return "MergeParts"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override { @@ -1651,7 +1651,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); - auto merge_step = std::make_unique( + auto merge_step = std::make_unique( merge_parts_query_plan.getCurrentDataStream(), sort_description, partition_key_columns, From 8c1f434b1ac2c9fbb83561a43a6ee10f20d81974 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 9 Sep 2024 15:31:43 +0200 Subject: [PATCH 27/40] Do column materialization using ActionsDAG::addMaterializingOutputActions instead of a special step --- src/Storages/MergeTree/MergeTask.cpp | 42 +--------------------------- 1 file changed, 1 insertion(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 3ca909a2d09..33cdff10b6a 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1456,45 +1456,6 @@ private: const time_t time_of_merge{0}; }; - -class MaterializingStep : public ITransformingStep -{ -public: - explicit MaterializingStep( - const DataStream & input_stream_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits()) - {} - - String getName() const override { return "Materializing"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override - { - pipeline.addTransform(std::make_shared(input_streams.front().header)); - } - - void updateOutputStream() override - { - output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - } - -private: - static Traits getTraits() - { - return ITransformingStep::Traits - { - { - .returns_single_stream = true, - .preserves_number_of_streams = true, - .preserves_sorting = true, - }, - { - .preserves_number_of_rows = true, - } - }; - } -}; - - class TTLStep : public ITransformingStep { public: @@ -1709,12 +1670,11 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const if (!global_ctx->merging_skip_indexes.empty()) { auto indices_expression_dag = global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization. auto calculate_indices_expression_step = std::make_unique( merge_parts_query_plan.getCurrentDataStream(), std::move(indices_expression_dag)); merge_parts_query_plan.addStep(std::move(calculate_indices_expression_step)); - /// TODO: what is the purpose of MaterializingTransform in the original code? - merge_parts_query_plan.addStep(std::make_unique(merge_parts_query_plan.getCurrentDataStream())); } if (!subqueries.empty()) From 4da1e10ac66059b47a89e69327ea79d487e2847f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 9 Sep 2024 16:01:00 +0200 Subject: [PATCH 28/40] Move sorting key calculation step outside the loop --- src/Storages/MergeTree/MergeTask.cpp | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 33cdff10b6a..9a1e749734c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1540,8 +1540,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); - auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone(); - /// Read from all parts std::vector plans; for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) @@ -1566,15 +1564,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const global_ctx->context, ctx->log); - if (global_ctx->metadata_snapshot->hasSortingKey()) - { - /// Calculate sorting key expressions so that they are available for merge sorting. - auto calculate_sorting_key_expression_step = std::make_unique( - plan_for_part->getCurrentDataStream(), - sorting_key_expression_dag.clone()); /// TODO: can we avoid cloning here? - plan_for_part->addStep(std::move(calculate_sorting_key_expression_step)); - } - plans.emplace_back(std::move(plan_for_part)); } @@ -1591,6 +1580,16 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); } + if (global_ctx->metadata_snapshot->hasSortingKey()) + { + /// Calculate sorting key expressions so that they are available for merge sorting. + auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone(); + auto calculate_sorting_key_expression_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + std::move(sorting_key_expression_dag)); + merge_parts_query_plan.addStep(std::move(calculate_sorting_key_expression_step)); + } + /// Merge { Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); From caab4dd8b903a8caaa45b0e9d811487110ac2560 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 18:20:48 +0200 Subject: [PATCH 29/40] fix --- src/Interpreters/Squashing.cpp | 2 +- .../03236_squashing_high_memory.reference | 0 .../03236_squashing_high_memory.sql | 22 +++++++++++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03236_squashing_high_memory.reference create mode 100644 tests/queries/0_stateless/03236_squashing_high_memory.sql diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 95b76c60063..c656a1a797b 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -45,7 +45,7 @@ Chunk Squashing::squash(Chunk && input_chunk) Chunk Squashing::add(Chunk && input_chunk) { - if (!input_chunk) + if (!input_chunk || input_chunk.getNumRows() == 0) return {}; /// Just read block is already enough. diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.reference b/tests/queries/0_stateless/03236_squashing_high_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql new file mode 100644 index 00000000000..523281fb74a --- /dev/null +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS id_values; + +DROP TABLE IF EXISTS test_table; + +CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS + SELECT arrayJoin(range(1000000)) AS id1, arrayJoin(range(1000)) AS id2; + +SET max_memory_usage = 1G; + +CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS +SELECT id_values.id1 AS id, + string_values.string_val1 AS string_val1, + string_values.string_val2 AS string_val2 +FROM id_values + JOIN (SELECT arrayJoin(range(10)) AS id1, + 'qwe' AS string_val1, + 'asd' AS string_val2) AS string_values + ON id_values.id1 = string_values.id1 + SETTINGS join_algorithm = 'hash'; + +DROP TABLE IF EXISTS id_values; +DROP TABLE IF EXISTS test_table; From df1821a5798b8d138c3da4b39517196ba06b447a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 10 Sep 2024 19:32:28 +0200 Subject: [PATCH 30/40] Update tests/queries/0_stateless/03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 523281fb74a..98b33b80e9f 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS - SELECT arrayJoin(range(1000000)) AS id1, arrayJoin(range(1000)) AS id2; + SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = 1G; From 9b517a939fce931da57a7d0604dc366ef35bf3fd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 19:53:16 +0200 Subject: [PATCH 31/40] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 98b33b80e9f..78316597430 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -5,7 +5,7 @@ DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; -SET max_memory_usage = 1G; +SET max_memory_usage = '1G'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, From 63aebfa5d7d410e1b33a1f914d02a8e75f52d5ea Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 20:15:36 +0200 Subject: [PATCH 32/40] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 78316597430..df296cb6713 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- reason: test requires too many rows to read + DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; From 2f15fcd23fb069a69d2f1a0caee36ae33f39fe45 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 10 Sep 2024 20:57:03 +0200 Subject: [PATCH 33/40] Test with sparse serialization, vertical merge and skip indices --- .../03175_sparse_and_skip_index.reference | 4 ++ .../03175_sparse_and_skip_index.sql | 45 +++++++++++++++++++ 2 files changed, 49 insertions(+) create mode 100644 tests/queries/0_stateless/03175_sparse_and_skip_index.reference create mode 100644 tests/queries/0_stateless/03175_sparse_and_skip_index.sql diff --git a/tests/queries/0_stateless/03175_sparse_and_skip_index.reference b/tests/queries/0_stateless/03175_sparse_and_skip_index.reference new file mode 100644 index 00000000000..619e98a152a --- /dev/null +++ b/tests/queries/0_stateless/03175_sparse_and_skip_index.reference @@ -0,0 +1,4 @@ +key Sparse +value Sparse +1000 +1 diff --git a/tests/queries/0_stateless/03175_sparse_and_skip_index.sql b/tests/queries/0_stateless/03175_sparse_and_skip_index.sql new file mode 100644 index 00000000000..4de6d1ac6df --- /dev/null +++ b/tests/queries/0_stateless/03175_sparse_and_skip_index.sql @@ -0,0 +1,45 @@ +DROP TABLE IF EXISTS t_bloom_filter; +CREATE TABLE t_bloom_filter( + key UInt64, + value UInt64, + + INDEX key_bf key TYPE bloom_filter(0.01) GRANULARITY 2147483648, -- bloom filter on sorting key column + INDEX value_bf value TYPE bloom_filter(0.01) GRANULARITY 2147483648 -- bloom filter on no-sorting column +) ENGINE=MergeTree ORDER BY key +SETTINGS + -- settings to trigger sparse serialization and vertical merge + ratio_of_defaults_for_sparse_serialization = 0.0 + ,vertical_merge_algorithm_min_rows_to_activate = 1 + ,vertical_merge_algorithm_min_columns_to_activate = 1 + ,allow_vertical_merges_from_compact_to_wide_parts = 1 + ,min_bytes_for_wide_part=0 +; + +SYSTEM STOP MERGES t_bloom_filter; + +-- Create at least one part +INSERT INTO t_bloom_filter +SELECT + number % 100 as key, -- 100 unique keys + rand() % 100 as value -- 100 unique values +FROM numbers(50_000); + +-- And another part +INSERT INTO t_bloom_filter +SELECT + number % 100 as key, -- 100 unique keys + rand() % 100 as value -- 100 unique values +FROM numbers(50_000, 50_000); + +SYSTEM START MERGES t_bloom_filter; + +-- Merge everything into a single part +OPTIMIZE TABLE t_bloom_filter FINAL; + +-- Check sparse serialization +SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_bloom_filter' AND active ORDER BY column; + +SELECT COUNT() FROM t_bloom_filter WHERE key = 1; + +-- Check bloom filter non-zero size +SELECT COUNT() FROM system.parts WHERE database = currentDatabase() AND table = 't_bloom_filter' AND secondary_indices_uncompressed_bytes > 200 AND active; From d4aa06524b4e65e2bca4ff851ad54dd4468e5cc0 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 10 Sep 2024 20:57:55 +0200 Subject: [PATCH 34/40] Add materialization when building indices in vertical merge --- src/Storages/MergeTree/MergeTask.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 9a1e749734c..5c993504245 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -952,6 +952,7 @@ MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::Vertic indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); auto indices_expression_dag = indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization. auto calculate_indices_expression_step = std::make_unique( merge_column_query_plan.getCurrentDataStream(), std::move(indices_expression_dag)); From 3d36f6dce36595e8796f3d7361ca8ffc1e4bab38 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 23:48:32 +0200 Subject: [PATCH 35/40] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index df296cb6713..479ea87f798 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -6,7 +6,7 @@ DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS - SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; + SELECT arrayJoin(range(20000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; From fd3bd785b880644862f1437b99557d6643d78504 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 11 Sep 2024 11:17:06 +0200 Subject: [PATCH 36/40] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 479ea87f798..e1c193c3ea6 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -1,12 +1,14 @@ -- Tags: no-fasttest -- reason: test requires too many rows to read +SET max_rows_to_read = '501G'; + DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS - SELECT arrayJoin(range(20000)) AS id1, arrayJoin(range(1000)) AS id2; + SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; From af469ffde471ff64e8e0201ceb96ba6e1a0a29c4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 11 Sep 2024 12:32:36 +0200 Subject: [PATCH 37/40] remove sanitizer builds --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index e1c193c3ea6..f6e5dbdef03 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-asan, no-tsan, no-msan, no-ubsan -- reason: test requires too many rows to read SET max_rows_to_read = '501G'; From 9e335abe1735eeed100c4b964bf24220b6086bf9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Sep 2024 13:05:24 +0000 Subject: [PATCH 38/40] CMake: Add comment about ICU data files --- contrib/icu-cmake/CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index afaa189701d..5714fef8347 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -481,6 +481,11 @@ if (ARCH_S390X) else() set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75l_dat.S" ) endif() +# ^^ you might be confused how for different little endian platforms (x86, ARM) the same assembly files can be used. +# These files are indeed assembly but they only contain data ('.long' directive), which makes them portable accross CPUs. +# Only the endianness and the character set (ASCII, EBCDIC) makes a difference, also see +# https://unicode-org.github.io/icu/userguide/icu_data/#sharing-icu-data-between-platforms, 'Sharing ICU Data Between Platforms') +# (and as an experiment, try re-generating the data files on x86 vs. ARM, ... you'll get exactly the same files) set(ICUDATA_SOURCES "${ICUDATA_SOURCE_FILE}" From abe8f29bf8d282951f30d47bf03d3d3fcc024a17 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 11 Sep 2024 17:32:20 +0200 Subject: [PATCH 39/40] Update test --- tests/integration/test_storage_s3_queue/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 9e3ee19179a..8b492018766 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -663,7 +663,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): ) except QueryRuntimeException as e: assert ( - "Table columns structure in ZooKeeper is different from local table structure" + "Existing table metadata in ZooKeeper differs in columns" in str(e) ) failed = True From c0a42ef877c7abf93db444c95f39a59cefe93d32 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 11 Sep 2024 15:42:04 +0000 Subject: [PATCH 40/40] Automatic style fix --- tests/integration/test_storage_s3_queue/test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 8b492018766..b75ad21f002 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -662,10 +662,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): }, ) except QueryRuntimeException as e: - assert ( - "Existing table metadata in ZooKeeper differs in columns" - in str(e) - ) + assert "Existing table metadata in ZooKeeper differs in columns" in str(e) failed = True assert failed is True