From fe8f808f40d3cf4a7eb66d756a56666f9bfbb35f Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Wed, 17 Jul 2024 22:04:45 +0800 Subject: [PATCH 01/43] perf: improve the loading of a great number of replicas from multiple disks --- src/common/replication.codes.h | 2 +- src/replica/replica_stub.cpp | 180 ++++++++++++++++++++------------- src/replica/replica_stub.h | 1 + src/utils/time_utils.h | 2 + 4 files changed, 116 insertions(+), 69 deletions(-) diff --git a/src/common/replication.codes.h b/src/common/replication.codes.h index a2c29ef458..5f0628026a 100644 --- a/src/common/replication.codes.h +++ b/src/common/replication.codes.h @@ -141,7 +141,6 @@ MAKE_EVENT_CODE(LPC_META_STATE_NORMAL, TASK_PRIORITY_COMMON) // THREAD_POOL_REPLICATION #define CURRENT_THREAD_POOL THREAD_POOL_REPLICATION -MAKE_EVENT_CODE(LPC_REPLICATION_INIT_LOAD, TASK_PRIORITY_COMMON) MAKE_EVENT_CODE(RPC_REPLICATION_WRITE_EMPTY, TASK_PRIORITY_COMMON) MAKE_EVENT_CODE(LPC_PER_REPLICA_CHECKPOINT_TIMER, TASK_PRIORITY_COMMON) MAKE_EVENT_CODE(LPC_PER_REPLICA_COLLECT_INFO_TIMER, TASK_PRIORITY_COMMON) @@ -186,6 +185,7 @@ MAKE_EVENT_CODE(LPC_REPLICATION_HIGH, TASK_PRIORITY_HIGH) // THREAD_POOL_LOCAL_APP #define CURRENT_THREAD_POOL THREAD_POOL_LOCAL_APP +MAKE_EVENT_CODE(LPC_REPLICATION_INIT_LOAD, TASK_PRIORITY_COMMON) MAKE_EVENT_CODE(LPC_WRITE, TASK_PRIORITY_COMMON) MAKE_EVENT_CODE(LPC_read_THROTTLING_DELAY, TASK_PRIORITY_COMMON) #undef CURRENT_THREAD_POOL diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 7a51ee4cac..f80f97b854 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -260,6 +260,19 @@ DSN_DECLARE_string(data_dirs); DSN_DECLARE_string(encryption_cluster_key_name); DSN_DECLARE_string(server_key); +DSN_DEFINE_uint64(replication, + max_replicas_on_load_for_each_disk, + 256, + "The max number of replicas that are allowed to be loaded simultaneously " + "for each disk dir."); + +/* +DSN_DEFINE_uint64(replication, + load_replicas_retry_interval_ms, + 50, + "The retry interval after max_replicas_on_load_for_each_disk has been reached."); + */ + DSN_DEFINE_bool(replication, deny_client_on_start, false, @@ -442,6 +455,95 @@ void replica_stub::initialize(bool clear /* = false*/) _access_controller = std::make_unique(); } +void replica_stub::load_replicas(replicas &reps) +{ + std::vector>> disks; + for (const auto &dn : _fs_manager.get_dir_nodes()) { + // Skip dir node with IO error. + if (dsn_unlikely(dn->status == disk_status::IO_ERROR)) { + continue; + } + + std::vector sub_dirs; + CHECK(dsn::utils::filesystem::get_subdirectories(dn->full_dir, sub_dirs, false), + "failed to get sub_directories in {}", + dn->full_dir); + disks.emplace_back(dn.get(), std::move(sub_dirs)); + } + + utils::ex_lock reps_lock; + std::vector> load_tasks; + load_tasks.reserve(disks.size()); + + std::vector threads; + + std::atomic task_hash(0); + for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { + load_tasks.emplace_back(); + + threads.emplace_back( + [this, &reps, &reps_lock, &task_hash, &load_tasks, &disks, disk_index]() mutable { + const auto &[dn, dirs] = disks[disk_index]; + + auto &disk_tasks = load_tasks[disk_index]; + for (size_t dir_index = 0; dir_index < dirs.size();) { + if (disk_tasks.size() >= FLAGS_max_replicas_on_load_for_each_disk) { + // std::this_thread::sleep_for( + // std::chrono::milliseconds(FLAGS_load_replicas_retry_interval_ms)); + disk_tasks.front()->wait(); + disk_tasks.pop_front(); + continue; + } + + const auto &dir = dirs[dir_index++]; + if (dsn::replication::is_data_dir_invalid(dir)) { + LOG_WARNING("ignore dir {}", dir); + continue; + } + + disk_tasks.push_back(tasking::create_task( + // Ensure that the thread pool is non-partitioned. + LPC_REPLICATION_INIT_LOAD, + &_tracker, + [this, dn, dir, &reps, &reps_lock] { + LOG_INFO("process dir {}", dir); + + auto r = load_replica(dn, dir.c_str()); + if (r == nullptr) { + return; + } + LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", + r->get_gpid(), + dsn_primary_host_port(), + dir, + r->last_durable_decree(), + r->last_committed_decree(), + r->last_prepared_decree()); + + utils::auto_lock l(reps_lock); + CHECK(reps.find(r->get_gpid()) == reps.end(), + "conflict replica dir: {} <--> {}", + r->dir(), + reps[r->get_gpid()]->dir()); + + reps[r->get_gpid()] = r; + }, + task_hash.fetch_add(1, std::memory_order_relaxed))); + disk_tasks.back()->enqueue(); + } + + for (auto &disk_task : disk_tasks) { + disk_task->wait(); + } + }); + } + + for (auto &t : threads) { + t.join(); + } +} + void replica_stub::initialize(const replication_options &opts, bool clear /* = false*/) { _primary_host_port = dsn_primary_host_port(); @@ -526,75 +628,17 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f // Start to load replicas in available data directories. LOG_INFO("start to load replicas"); - std::map> dirs_by_dn; - for (const auto &dn : _fs_manager.get_dir_nodes()) { - // Skip IO error dir_node. - if (dsn_unlikely(dn->status == disk_status::IO_ERROR)) { - continue; - } - std::vector sub_directories; - CHECK(dsn::utils::filesystem::get_subdirectories(dn->full_dir, sub_directories, false), - "fail to get sub_directories in {}", - dn->full_dir); - dirs_by_dn.emplace(dn.get(), sub_directories); - } - - replicas rps; - utils::ex_lock rps_lock; - std::deque load_tasks; - uint64_t start_time = dsn_now_ms(); - for (const auto &dn_dirs : dirs_by_dn) { - const auto dn = dn_dirs.first; - for (const auto &dir : dn_dirs.second) { - if (dsn::replication::is_data_dir_invalid(dir)) { - LOG_WARNING("ignore dir {}", dir); - continue; - } - load_tasks.push_back(tasking::create_task( - LPC_REPLICATION_INIT_LOAD, - &_tracker, - [this, dn, dir, &rps, &rps_lock] { - LOG_INFO("process dir {}", dir); - - auto r = load_replica(dn, dir.c_str()); - if (r == nullptr) { - return; - } - LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", - r->get_gpid(), - dsn_primary_host_port(), - dir, - r->last_durable_decree(), - r->last_committed_decree(), - r->last_prepared_decree()); - - utils::auto_lock l(rps_lock); - CHECK(rps.find(r->get_gpid()) == rps.end(), - "conflict replica dir: {} <--> {}", - r->dir(), - rps[r->get_gpid()]->dir()); - - rps[r->get_gpid()] = r; - }, - load_tasks.size())); - load_tasks.back()->enqueue(); - } - } - for (auto &tsk : load_tasks) { - tsk->wait(); - } - uint64_t finish_time = dsn_now_ms(); + replicas reps; + utils::chronograph chrono; + load_replicas(reps); - dirs_by_dn.clear(); - load_tasks.clear(); LOG_INFO("load replicas succeed, replica_count = {}, time_used = {} ms", - rps.size(), - finish_time - start_time); + reps.size(), + chrono.duration_ms()); bool is_log_complete = true; - for (auto it = rps.begin(); it != rps.end(); ++it) { + for (auto it = reps.begin(); it != reps.end(); ++it) { CHECK_EQ_MSG(it->second->background_sync_checkpoint(), ERR_OK, "sync checkpoint failed"); it->second->reset_prepare_list_after_replay(); @@ -624,7 +668,7 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f if (!is_log_complete) { LOG_ERROR("logs are not complete for some replicas, which means that shared log is " "truncated, mark all replicas as inactive"); - for (auto it = rps.begin(); it != rps.end(); ++it) { + for (auto it = reps.begin(); it != reps.end(); ++it) { it->second->set_inactive_state_transient(false); } } @@ -651,8 +695,8 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f std::chrono::seconds(FLAGS_disk_stat_interval_seconds)); } - // attach rps - _replicas = std::move(rps); + // attach reps + _replicas = std::move(reps); METRIC_VAR_INCREMENT_BY(total_replicas, _replicas.size()); for (const auto &kv : _replicas) { _fs_manager.add_replica(kv.first, kv.second->dir()); @@ -1390,7 +1434,7 @@ void replica_stub::on_node_query_reply(error_code err, it->config.pid.thread_hash()); } - // for rps not exist on meta_servers + // For the replicas that do not exist on meta_servers. for (auto it = rs.begin(); it != rs.end(); ++it) { tasking::enqueue( LPC_QUERY_NODE_CONFIGURATION_SCATTER2, diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index 328e975024..2aa0cb5907 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -143,6 +143,7 @@ class replica_stub : public serverlet, public ref_counter // // initialization // + void load_replicas(replicas &reps); void initialize(const replication_options &opts, bool clear = false); void initialize(bool clear = false); void set_options(const replication_options &opts) { _options = opts; } diff --git a/src/utils/time_utils.h b/src/utils/time_utils.h index d36aa98640..8730b5d7f4 100644 --- a/src/utils/time_utils.h +++ b/src/utils/time_utils.h @@ -155,6 +155,8 @@ class chronograph return now - _start_time_ns; } + inline uint64_t duration_ms() const { return duration_ns() / 1'000'000; } + private: uint64_t _start_time_ns; From 5d5b74c7ffb1889ea6fbe88f9bd869206331d2ed Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 19 Jul 2024 00:57:13 +0800 Subject: [PATCH 02/43] perf: improve the loading of a great number of replicas from multiple disks --- src/replica/replica_stub.cpp | 157 ++++++++++++++++------------ src/utils/simple_concurrent_queue.h | 88 ++++++++++++++++ 2 files changed, 181 insertions(+), 64 deletions(-) create mode 100644 src/utils/simple_concurrent_queue.h diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index f80f97b854..dfd18a2402 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -80,6 +80,7 @@ #include "utils/ports.h" #include "utils/process_utils.h" #include "utils/rand.h" +#include "utils/simple_concurrent_queue.h" #include "utils/strings.h" #include "utils/synchronize.h" #ifdef DSN_ENABLE_GPERF @@ -266,12 +267,10 @@ DSN_DEFINE_uint64(replication, "The max number of replicas that are allowed to be loaded simultaneously " "for each disk dir."); -/* DSN_DEFINE_uint64(replication, load_replicas_retry_interval_ms, - 50, + 10, "The retry interval after max_replicas_on_load_for_each_disk has been reached."); - */ DSN_DEFINE_bool(replication, deny_client_on_start, @@ -471,76 +470,106 @@ void replica_stub::load_replicas(replicas &reps) disks.emplace_back(dn.get(), std::move(sub_dirs)); } - utils::ex_lock reps_lock; - std::vector> load_tasks; - load_tasks.reserve(disks.size()); + std::vector>> load_disk_queues; + load_disk_queues.reserve(disks.size()); + for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { + load_disk_queues.push_back(std::make_unique>()); + } std::vector threads; - - std::atomic task_hash(0); for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - load_tasks.emplace_back(); - - threads.emplace_back( - [this, &reps, &reps_lock, &task_hash, &load_tasks, &disks, disk_index]() mutable { - const auto &[dn, dirs] = disks[disk_index]; - - auto &disk_tasks = load_tasks[disk_index]; - for (size_t dir_index = 0; dir_index < dirs.size();) { - if (disk_tasks.size() >= FLAGS_max_replicas_on_load_for_each_disk) { - // std::this_thread::sleep_for( - // std::chrono::milliseconds(FLAGS_load_replicas_retry_interval_ms)); - disk_tasks.front()->wait(); - disk_tasks.pop_front(); - continue; - } + threads.emplace_back([&load_disk_queues, &disks, disk_index]() mutable { + auto &load_disk_queue = load_disk_queues[disk_index]; + while (true) { + task_ptr load_replica_task; + load_disk_queue->pop(load_replica_task); + if (load_replica_task == nullptr) { + break; + } + + load_replica_task->wait(); + } + }); + } + + utils::ex_lock reps_lock; + std::vector dir_indexes(disks.size(), 0); + + while (true) { + size_t finished_disks = 0; + size_t throttling_disks = 0; + + for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { + auto &load_disk_queue = load_disk_queues[disk_index]; + auto &dir_index = dir_indexes[disk_index]; + const auto &[dn, dirs] = disks[disk_index]; + + if (dir_index >= dirs.size()) { + ++finished_disks; + continue; + } + + if (load_disk_queue->size() >= FLAGS_max_replicas_on_load_for_each_disk) { + ++throttling_disks; + continue; + } + + const auto &dir = dirs[dir_index++]; + if (dsn::replication::is_data_dir_invalid(dir)) { + LOG_WARNING("ignore dir {}", dir); + continue; + } - const auto &dir = dirs[dir_index++]; - if (dsn::replication::is_data_dir_invalid(dir)) { - LOG_WARNING("ignore dir {}", dir); - continue; + auto load_replica_task = tasking::create_task( + // Ensure that the thread pool is non-partitioned. + LPC_REPLICATION_INIT_LOAD, + &_tracker, + [this, dn, dir, &reps, &reps_lock] { + LOG_INFO("process dir {}", dir); + + auto r = load_replica(dn, dir.c_str()); + if (r == nullptr) { + return; } - disk_tasks.push_back(tasking::create_task( - // Ensure that the thread pool is non-partitioned. - LPC_REPLICATION_INIT_LOAD, - &_tracker, - [this, dn, dir, &reps, &reps_lock] { - LOG_INFO("process dir {}", dir); - - auto r = load_replica(dn, dir.c_str()); - if (r == nullptr) { - return; - } - LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", - r->get_gpid(), - dsn_primary_host_port(), - dir, - r->last_durable_decree(), - r->last_committed_decree(), - r->last_prepared_decree()); - - utils::auto_lock l(reps_lock); - CHECK(reps.find(r->get_gpid()) == reps.end(), - "conflict replica dir: {} <--> {}", - r->dir(), - reps[r->get_gpid()]->dir()); - - reps[r->get_gpid()] = r; - }, - task_hash.fetch_add(1, std::memory_order_relaxed))); - disk_tasks.back()->enqueue(); - } + LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", + r->get_gpid(), + dsn_primary_host_port(), + dir, + r->last_durable_decree(), + r->last_committed_decree(), + r->last_prepared_decree()); + + utils::auto_lock l(reps_lock); + CHECK(reps.find(r->get_gpid()) == reps.end(), + "conflict replica dir: {} <--> {}", + r->dir(), + reps[r->get_gpid()]->dir()); + + reps[r->get_gpid()] = r; + }); - for (auto &disk_task : disk_tasks) { - disk_task->wait(); - } - }); + load_replica_task->enqueue(); + load_disk_queue->push(std::move(load_replica_task)); + } + + if (finished_disks >= disks.size()) { + break; + } + + if (throttling_disks >= disks.size()) { + std::this_thread::sleep_for( + std::chrono::milliseconds(FLAGS_load_replicas_retry_interval_ms)); + } + } + + for (auto &load_disk_queue : load_disk_queues) { + load_disk_queue->push(task_ptr()); } - for (auto &t : threads) { - t.join(); + for (auto &thread : threads) { + thread.join(); } } diff --git a/src/utils/simple_concurrent_queue.h b/src/utils/simple_concurrent_queue.h new file mode 100644 index 0000000000..835a9add8b --- /dev/null +++ b/src/utils/simple_concurrent_queue.h @@ -0,0 +1,88 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include + +#include "utils/ports.h" + +namespace dsn { + +template +class simple_concurrent_queue +{ +public: + using value_type = T; + + simple_concurrent_queue() = default; + + ~simple_concurrent_queue() = default; + + void push(value_type &&value) + { + { + std::lock_guard lock(_mtx); + _queue.push(std::forward(value)); + } + + _cond_var.notify_one(); + } + + void pop(value_type &value) + { + std::unique_lock lock(_mtx); + _cond_var.wait(lock, [this] { return !_queue.empty(); }); + + value = _queue.front(); + _queue.pop(); + } + + bool pop(uint64_t timeout_ns, value_type &value) + { + std::unique_lock lock(_mtx); + const auto status = _cond_var.wait_for( + lock, std::chrono::nanoseconds(timeout_ns), [this] { return !_queue.empty(); }); + + if (status == std::cv_status::timeout) { + return false; + } + + value = _queue.front(); + _queue.pop(); + return true; + } + + size_t size() const + { + std::lock_guard lock(_mtx); + return _queue.size(); + } + +private: + std::queue _queue; + mutable std::mutex _mtx; + mutable std::condition_variable _cond_var; + + DISALLOW_COPY_AND_ASSIGN(simple_concurrent_queue); +}; + +} // namespace dsn From a13dac481aea74f46f653e93dbb9bf4058cf8b2e Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 19 Jul 2024 11:30:47 +0800 Subject: [PATCH 03/43] fix compilation --- src/replica/replica_stub.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index dfd18a2402..5341cc6ebc 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -478,7 +478,7 @@ void replica_stub::load_replicas(replicas &reps) std::vector threads; for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - threads.emplace_back([&load_disk_queues, &disks, disk_index]() mutable { + threads.emplace_back([&load_disk_queues, disk_index]() mutable { auto &load_disk_queue = load_disk_queues[disk_index]; while (true) { task_ptr load_replica_task; @@ -500,15 +500,14 @@ void replica_stub::load_replicas(replicas &reps) size_t throttling_disks = 0; for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - auto &load_disk_queue = load_disk_queues[disk_index]; auto &dir_index = dir_indexes[disk_index]; - const auto &[dn, dirs] = disks[disk_index]; - + const auto &dirs = disks[disk_index].second; if (dir_index >= dirs.size()) { ++finished_disks; continue; } + auto &load_disk_queue = load_disk_queues[disk_index]; if (load_disk_queue->size() >= FLAGS_max_replicas_on_load_for_each_disk) { ++throttling_disks; continue; @@ -520,6 +519,16 @@ void replica_stub::load_replicas(replicas &reps) continue; } + // Structured bindings can be captured by closures in g++, while not supported + // well by clang. Thus we do not use following statement to bind both variables + // until clang has been upgraded to version 16 which could support that well: + // + // const auto &[dn, dirs] = disks[disk_index]; + // + // For the docs of clang 16 please see: + // + // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support: + const auto &dn = disks[disk_index].first; auto load_replica_task = tasking::create_task( // Ensure that the thread pool is non-partitioned. LPC_REPLICATION_INIT_LOAD, From ec3f247c31efdb1f4f9ca9cde9509442dfb0be75 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 19 Jul 2024 19:19:14 +0800 Subject: [PATCH 04/43] fix tests --- src/replica/replica_stub.cpp | 59 ++++++++++++++++++++++++++---------- 1 file changed, 43 insertions(+), 16 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 5341cc6ebc..372f3b0828 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -35,7 +35,7 @@ #include #include #include -#include +#include #include #include #include @@ -83,6 +83,7 @@ #include "utils/simple_concurrent_queue.h" #include "utils/strings.h" #include "utils/synchronize.h" +#include "utils/time_utils.h" #ifdef DSN_ENABLE_GPERF #include #elif defined(DSN_USE_JEMALLOC) @@ -268,7 +269,7 @@ DSN_DEFINE_uint64(replication, "for each disk dir."); DSN_DEFINE_uint64(replication, - load_replicas_retry_interval_ms, + wait_load_replica_ms, 10, "The retry interval after max_replicas_on_load_for_each_disk has been reached."); @@ -464,40 +465,52 @@ void replica_stub::load_replicas(replicas &reps) } std::vector sub_dirs; + LOG_INFO("full dir {}", dn->full_dir); CHECK(dsn::utils::filesystem::get_subdirectories(dn->full_dir, sub_dirs, false), "failed to get sub_directories in {}", dn->full_dir); disks.emplace_back(dn.get(), std::move(sub_dirs)); } - std::vector>> load_disk_queues; - load_disk_queues.reserve(disks.size()); + std::vector> load_disk_queues(disks.size()); + /* load_disk_queues.reserve(disks.size()); for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { load_disk_queues.push_back(std::make_unique>()); - } + } */ + + /* + const auto &service_info = service_app::current_service_app_info(); std::vector threads; for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - threads.emplace_back([&load_disk_queues, disk_index]() mutable { + threads.emplace_back([&service_info, &load_disk_queues, disk_index]() mutable { + // Initialize non-dsn thread. + LOG_INFO("role_name={}, index={}",service_info.role_name.c_str(), service_info.index); + dsn_mimic_app(service_info.role_name.c_str(), service_info.index); + auto &load_disk_queue = load_disk_queues[disk_index]; while (true) { task_ptr load_replica_task; + LOG_INFO("to pop"); load_disk_queue->pop(load_replica_task); if (load_replica_task == nullptr) { + LOG_INFO("load_replica_task null"); break; } + LOG_INFO("load_replica_task not null"); load_replica_task->wait(); + LOG_INFO("after wait"); } }); } + */ utils::ex_lock reps_lock; std::vector dir_indexes(disks.size(), 0); while (true) { size_t finished_disks = 0; - size_t throttling_disks = 0; for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { auto &dir_index = dir_indexes[disk_index]; @@ -508,12 +521,15 @@ void replica_stub::load_replicas(replicas &reps) } auto &load_disk_queue = load_disk_queues[disk_index]; - if (load_disk_queue->size() >= FLAGS_max_replicas_on_load_for_each_disk) { - ++throttling_disks; + if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { + if (load_disk_queue.front()->wait(FLAGS_wait_load_replica_ms)) { + load_disk_queue.pop(); + } continue; } const auto &dir = dirs[dir_index++]; + LOG_INFO("load dir {}", dir); if (dsn::replication::is_data_dir_invalid(dir)) { LOG_WARNING("ignore dir {}", dir); continue; @@ -529,7 +545,7 @@ void replica_stub::load_replicas(replicas &reps) // // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support: const auto &dn = disks[disk_index].first; - auto load_replica_task = tasking::create_task( + load_disk_queue.push(tasking::create_task( // Ensure that the thread pool is non-partitioned. LPC_REPLICATION_INIT_LOAD, &_tracker, @@ -538,6 +554,7 @@ void replica_stub::load_replicas(replicas &reps) auto r = load_replica(dn, dir.c_str()); if (r == nullptr) { + LOG_INFO("process dir null"); return; } @@ -557,29 +574,35 @@ void replica_stub::load_replicas(replicas &reps) reps[r->get_gpid()]->dir()); reps[r->get_gpid()] = r; - }); + })); - load_replica_task->enqueue(); - load_disk_queue->push(std::move(load_replica_task)); + load_disk_queue.back()->enqueue(); } + LOG_INFO("finished disks: {}, disk size = {}", finished_disks, disks.size()); if (finished_disks >= disks.size()) { break; } + } - if (throttling_disks >= disks.size()) { - std::this_thread::sleep_for( - std::chrono::milliseconds(FLAGS_load_replicas_retry_interval_ms)); + for (auto &load_disk_queue : load_disk_queues) { + while (!load_disk_queue.empty()) { + CHECK_TRUE(load_disk_queue.front()->wait()); + load_disk_queue.pop(); } } + /* for (auto &load_disk_queue : load_disk_queues) { load_disk_queue->push(task_ptr()); } + LOG_INFO("notify finish"); for (auto &thread : threads) { thread.join(); } + LOG_INFO("load done"); + */ } void replica_stub::initialize(const replication_options &opts, bool clear /* = false*/) @@ -2124,6 +2147,8 @@ replica *replica_stub::load_replica(dir_node *dn, const char *dir) FAIL_POINT_INJECT_F("mock_replica_load", [&](std::string_view) -> replica * { return nullptr; }); + LOG_INFO("{}: begin validate replica", dir); + app_info ai; gpid pid; std::string hint_message; @@ -2135,7 +2160,9 @@ replica *replica_stub::load_replica(dir_node *dn, const char *dir) // The replica's directory must exist when creating a replica. CHECK_EQ(dir, dn->replica_dir(ai.app_type, pid)); auto *rep = new replica(this, pid, ai, dn, false); + LOG_INFO("{}: begin load replica", rep->name()); const auto err = rep->initialize_on_load(); + LOG_INFO("{}: end load replica", rep->name()); if (err != ERR_OK) { LOG_ERROR("{}: load replica failed, err = {}", rep->name(), err); rep->close(); From ea00a109c21e047d5cb00212f65964959948e3de Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 19 Jul 2024 19:30:20 +0800 Subject: [PATCH 05/43] format --- src/replica/replica_stub.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 372f3b0828..2231f31ddf 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -478,7 +478,7 @@ void replica_stub::load_replicas(replicas &reps) load_disk_queues.push_back(std::make_unique>()); } */ - /* + /* const auto &service_info = service_app::current_service_app_info(); std::vector threads; From 6bbeefbbf32ec45723d0b2d8a3d923b9ce528728 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Sat, 20 Jul 2024 00:58:26 +0800 Subject: [PATCH 06/43] fix IWYU --- src/replica/replica_stub.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 2231f31ddf..88ae04c807 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -35,9 +35,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -80,7 +80,6 @@ #include "utils/ports.h" #include "utils/process_utils.h" #include "utils/rand.h" -#include "utils/simple_concurrent_queue.h" #include "utils/strings.h" #include "utils/synchronize.h" #include "utils/time_utils.h" From c43db34bf0bd78a4b53de08fe24950eb9f6f1917 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Sat, 20 Jul 2024 01:31:40 +0800 Subject: [PATCH 07/43] refactor --- src/replica/replica_stub.cpp | 77 +++++++----------------------------- 1 file changed, 14 insertions(+), 63 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 88ae04c807..6c5c095c64 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -268,9 +268,9 @@ DSN_DEFINE_uint64(replication, "for each disk dir."); DSN_DEFINE_uint64(replication, - wait_load_replica_ms, + load_replica_max_wait_time_ms, 10, - "The retry interval after max_replicas_on_load_for_each_disk has been reached."); + "The max waiting time for replica loading to complete."); DSN_DEFINE_bool(replication, deny_client_on_start, @@ -464,49 +464,15 @@ void replica_stub::load_replicas(replicas &reps) } std::vector sub_dirs; - LOG_INFO("full dir {}", dn->full_dir); CHECK(dsn::utils::filesystem::get_subdirectories(dn->full_dir, sub_dirs, false), "failed to get sub_directories in {}", dn->full_dir); disks.emplace_back(dn.get(), std::move(sub_dirs)); } + std::vector dir_indexes(disks.size(), 0); std::vector> load_disk_queues(disks.size()); - /* load_disk_queues.reserve(disks.size()); - for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - load_disk_queues.push_back(std::make_unique>()); - } */ - - /* - const auto &service_info = service_app::current_service_app_info(); - - std::vector threads; - for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - threads.emplace_back([&service_info, &load_disk_queues, disk_index]() mutable { - // Initialize non-dsn thread. - LOG_INFO("role_name={}, index={}",service_info.role_name.c_str(), service_info.index); - dsn_mimic_app(service_info.role_name.c_str(), service_info.index); - - auto &load_disk_queue = load_disk_queues[disk_index]; - while (true) { - task_ptr load_replica_task; - LOG_INFO("to pop"); - load_disk_queue->pop(load_replica_task); - if (load_replica_task == nullptr) { - LOG_INFO("load_replica_task null"); - break; - } - LOG_INFO("load_replica_task not null"); - - load_replica_task->wait(); - LOG_INFO("after wait"); - } - }); - } - */ - utils::ex_lock reps_lock; - std::vector dir_indexes(disks.size(), 0); while (true) { size_t finished_disks = 0; @@ -521,14 +487,13 @@ void replica_stub::load_replicas(replicas &reps) auto &load_disk_queue = load_disk_queues[disk_index]; if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { - if (load_disk_queue.front()->wait(FLAGS_wait_load_replica_ms)) { + if (load_disk_queue.front()->wait(FLAGS_load_replica_max_wait_time_ms)) { load_disk_queue.pop(); } continue; } const auto &dir = dirs[dir_index++]; - LOG_INFO("load dir {}", dir); if (dsn::replication::is_data_dir_invalid(dir)) { LOG_WARNING("ignore dir {}", dir); continue; @@ -551,34 +516,32 @@ void replica_stub::load_replicas(replicas &reps) [this, dn, dir, &reps, &reps_lock] { LOG_INFO("process dir {}", dir); - auto r = load_replica(dn, dir.c_str()); - if (r == nullptr) { - LOG_INFO("process dir null"); + auto rep = load_replica(dn, dir.c_str()); + if (rep == nullptr) { return; } LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", - r->get_gpid(), + rep->get_gpid(), dsn_primary_host_port(), dir, - r->last_durable_decree(), - r->last_committed_decree(), - r->last_prepared_decree()); + rep->last_durable_decree(), + rep->last_committed_decree(), + rep->last_prepared_decree()); utils::auto_lock l(reps_lock); - CHECK(reps.find(r->get_gpid()) == reps.end(), + CHECK(reps.find(rep->get_gpid()) == reps.end(), "conflict replica dir: {} <--> {}", - r->dir(), - reps[r->get_gpid()]->dir()); + rep->dir(), + reps[rep->get_gpid()]->dir()); - reps[r->get_gpid()] = r; + reps[rep->get_gpid()] = rep; })); load_disk_queue.back()->enqueue(); } - LOG_INFO("finished disks: {}, disk size = {}", finished_disks, disks.size()); if (finished_disks >= disks.size()) { break; } @@ -590,18 +553,6 @@ void replica_stub::load_replicas(replicas &reps) load_disk_queue.pop(); } } - - /* - for (auto &load_disk_queue : load_disk_queues) { - load_disk_queue->push(task_ptr()); - } - - LOG_INFO("notify finish"); - for (auto &thread : threads) { - thread.join(); - } - LOG_INFO("load done"); - */ } void replica_stub::initialize(const replication_options &opts, bool clear /* = false*/) From 1f64a4f4932b640683c6bfbfb3e7c1c1e9384357 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Wed, 24 Jul 2024 12:42:33 +0800 Subject: [PATCH 08/43] refactor --- src/replica/replica_stub.cpp | 74 +++++++++++++++++++++++------------- src/replica/replica_stub.h | 9 ++++- 2 files changed, 55 insertions(+), 28 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 6c5c095c64..0ad5700c91 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -454,9 +454,9 @@ void replica_stub::initialize(bool clear /* = false*/) _access_controller = std::make_unique(); } -void replica_stub::load_replicas(replicas &reps) +replica_stub::disk_dirs replica_stub::get_all_disk_dirs() const { - std::vector>> disks; + disk_dirs disks; for (const auto &dn : _fs_manager.get_dir_nodes()) { // Skip dir node with IO error. if (dsn_unlikely(dn->status == disk_status::IO_ERROR)) { @@ -470,6 +470,43 @@ void replica_stub::load_replicas(replicas &reps) disks.emplace_back(dn.get(), std::move(sub_dirs)); } + return disks; +} + +void replica_stub::load_replica(dir_node *dn, + const std::string &dir, + utils::ex_lock &reps_lock, + replicas &reps) +{ + LOG_INFO("process dir {}", dir); + + auto rep = load_replica(dn, dir.c_str()); + if (rep == nullptr) { + return; + } + + LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", + rep->get_gpid(), + dsn_primary_host_port(), + dir, + rep->last_durable_decree(), + rep->last_committed_decree(), + rep->last_prepared_decree()); + + utils::auto_lock l(reps_lock); + CHECK(reps.find(rep->get_gpid()) == reps.end(), + "conflict replica dir: {} <--> {}", + rep->dir(), + reps[rep->get_gpid()]->dir()); + + reps[rep->get_gpid()] = rep; +} + +void replica_stub::load_replicas(replicas &reps) +{ + const auto &disks = get_all_disk_dirs(); + std::vector dir_indexes(disks.size(), 0); std::vector> load_disk_queues(disks.size()); utils::ex_lock reps_lock; @@ -513,31 +550,14 @@ void replica_stub::load_replicas(replicas &reps) // Ensure that the thread pool is non-partitioned. LPC_REPLICATION_INIT_LOAD, &_tracker, - [this, dn, dir, &reps, &reps_lock] { - LOG_INFO("process dir {}", dir); - - auto rep = load_replica(dn, dir.c_str()); - if (rep == nullptr) { - return; - } - - LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", - rep->get_gpid(), - dsn_primary_host_port(), - dir, - rep->last_durable_decree(), - rep->last_committed_decree(), - rep->last_prepared_decree()); - - utils::auto_lock l(reps_lock); - CHECK(reps.find(rep->get_gpid()) == reps.end(), - "conflict replica dir: {} <--> {}", - rep->dir(), - reps[rep->get_gpid()]->dir()); - - reps[rep->get_gpid()] = rep; - })); + std::bind(static_cast( + &replica_stub::load_replica), + this, + dn, + dir, + std::ref(reps_lock), + std::ref(reps)))); load_disk_queue.back()->enqueue(); } diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index 2aa0cb5907..c61ba5b332 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -143,7 +143,6 @@ class replica_stub : public serverlet, public ref_counter // // initialization // - void load_replicas(replicas &reps); void initialize(const replication_options &opts, bool clear = false); void initialize(bool clear = false); void set_options(const replication_options &opts) { _options = opts; } @@ -337,6 +336,13 @@ class replica_stub : public serverlet, public ref_counter RL_closed }; + using disk_dirs = std::vector>>; + disk_dirs get_all_disk_dirs() const; + + void + load_replica(dir_node *dn, const std::string &dir, utils::ex_lock &reps_lock, replicas &reps); + void load_replicas(replicas &reps); + void initialize_start(); void query_configuration_by_node(); void on_meta_server_disconnected_scatter(replica_stub_ptr this_, gpid id); @@ -445,6 +451,7 @@ class replica_stub : public serverlet, public ref_counter friend class replica_follower; friend class replica_follower_test; friend class replica_http_service_test; + friend class load_replicas_test; FRIEND_TEST(open_replica_test, open_replica_add_decree_and_ballot_check); FRIEND_TEST(replica_test, test_auto_trash_of_corruption); FRIEND_TEST(replica_test, test_clear_on_failure); From e2379b468af20775cdf9586ea299149f167cdb71 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Wed, 24 Jul 2024 19:42:56 +0800 Subject: [PATCH 09/43] add test for loading replicas --- src/replica/replica.h | 1 + src/replica/replica_stub.cpp | 41 ++++--- src/replica/replica_stub.h | 15 ++- src/replica/test/config-test.ini | 8 +- src/replica/test/load_replicas_test.cpp | 137 ++++++++++++++++++++++++ src/utils/autoref_ptr.h | 9 ++ 6 files changed, 194 insertions(+), 17 deletions(-) create mode 100644 src/replica/test/load_replicas_test.cpp diff --git a/src/replica/replica.h b/src/replica/replica.h index 4ee215c3dc..a6bcfe4086 100644 --- a/src/replica/replica.h +++ b/src/replica/replica.h @@ -610,6 +610,7 @@ class replica : public serverlet, public ref_counter, public replica_ba friend class replica_disk_test; friend class replica_disk_migrate_test; friend class open_replica_test; + friend class LoadReplicasTest; friend class replica_follower; friend class ::pegasus::server::pegasus_server_test_base; friend class ::pegasus::server::rocksdb_wrapper_test; diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 0ad5700c91..0b9b21d4fd 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -2059,6 +2059,27 @@ replica *replica_stub::new_replica(gpid gpid, return rep; } +/*static*/ std::string replica_stub::get_replica_dir_name(const std::string &dir) +{ + static const char splitters[] = {'\\', '/', 0}; + return utils::get_last_component(dir, splitters); +} + +/* static */ bool +replica_stub::parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type) +{ + int32_t app_id, partition_id; + char app_type_buf[128] = {0}; + if (3 != sscanf(dir_name.c_str(), "%d.%d.%s", &app_id, &partition_id, app_type_buf)) { + return false; + } + + pid.set_app_id(app_id); + pid.set_partition_index(partition_id); + app_type = app_type_buf; + return true; +} + bool replica_stub::validate_replica_dir(const std::string &dir, app_info &ai, gpid &pid, @@ -2069,21 +2090,18 @@ bool replica_stub::validate_replica_dir(const std::string &dir, return false; } - char splitters[] = {'\\', '/', 0}; - const auto name = utils::get_last_component(dir, splitters); - if (name.empty()) { + const auto &dir_name = get_replica_dir_name(dir); + if (dir_name.empty()) { hint_message = fmt::format("invalid replica dir '{}'", dir); return false; } - char app_type[128] = {0}; - int32_t app_id, pidx; - if (3 != sscanf(name.c_str(), "%d.%d.%s", &app_id, &pidx, app_type)) { + std::string app_type; + if (!parse_replica_dir_name(dir_name, pid, app_type)) { hint_message = fmt::format("invalid replica dir '{}'", dir); return false; } - pid = gpid(app_id, pidx); replica_app_info rai(&ai); const auto ai_path = utils::filesystem::path_combine(dir, replica_app_info::kAppInfo); const auto err = rai.load(ai_path); @@ -2100,7 +2118,7 @@ bool replica_stub::validate_replica_dir(const std::string &dir, // When the online partition split function aborted, the garbage partitions are with pidx in // the range of [ai.partition_count, 2 * ai.partition_count), which means the partitions with // pidx >= ai.partition_count are garbage partitions. - if (ai.partition_count <= pidx) { + if (ai.partition_count <= pid.get_partition_index()) { hint_message = fmt::format( "partition[{}], count={}, this replica may be partition split garbage partition, " "ignore it", @@ -2112,13 +2130,11 @@ bool replica_stub::validate_replica_dir(const std::string &dir, return true; } -replica *replica_stub::load_replica(dir_node *dn, const char *dir) +replica_ptr replica_stub::load_replica(dir_node *dn, const char *dir) { FAIL_POINT_INJECT_F("mock_replica_load", [&](std::string_view) -> replica * { return nullptr; }); - LOG_INFO("{}: begin validate replica", dir); - app_info ai; gpid pid; std::string hint_message; @@ -2130,12 +2146,9 @@ replica *replica_stub::load_replica(dir_node *dn, const char *dir) // The replica's directory must exist when creating a replica. CHECK_EQ(dir, dn->replica_dir(ai.app_type, pid)); auto *rep = new replica(this, pid, ai, dn, false); - LOG_INFO("{}: begin load replica", rep->name()); const auto err = rep->initialize_on_load(); - LOG_INFO("{}: end load replica", rep->name()); if (err != ERR_OK) { LOG_ERROR("{}: load replica failed, err = {}", rep->name(), err); - rep->close(); delete rep; rep = nullptr; diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index c61ba5b332..a54641b6a7 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -28,6 +28,7 @@ #include #include +#include #include #include #include @@ -70,6 +71,12 @@ #include "utils/metrics.h" #include "utils/zlocks.h" +namespace dsn { +namespace utils { +class ex_lock; +} // namespace utils +} // namespace dsn + DSN_DECLARE_uint32(max_concurrent_manual_emergency_checkpointing_count); namespace dsn { @@ -339,6 +346,10 @@ class replica_stub : public serverlet, public ref_counter using disk_dirs = std::vector>>; disk_dirs get_all_disk_dirs() const; + static std::string get_replica_dir_name(const std::string &dir); + static bool + parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type); + void load_replica(dir_node *dn, const std::string &dir, utils::ex_lock &reps_lock, replicas &reps); void load_replicas(replicas &reps); @@ -367,7 +378,7 @@ class replica_stub : public serverlet, public ref_counter bool is_duplication_follower, const std::string &parent_dir = ""); // Load an existing replica which is located in 'dn' with 'dir' directory. - replica *load_replica(dir_node *dn, const char *dir); + virtual replica_ptr load_replica(dir_node *dn, const char *dir); // Clean up the memory state and on disk data if creating replica failed. void clear_on_failure(replica *rep); task_ptr begin_close_replica(replica_ptr r); @@ -451,7 +462,7 @@ class replica_stub : public serverlet, public ref_counter friend class replica_follower; friend class replica_follower_test; friend class replica_http_service_test; - friend class load_replicas_test; + friend class LoadReplicasTest; FRIEND_TEST(open_replica_test, open_replica_add_decree_and_ballot_check); FRIEND_TEST(replica_test, test_auto_trash_of_corruption); FRIEND_TEST(replica_test, test_clear_on_failure); diff --git a/src/replica/test/config-test.ini b/src/replica/test/config-test.ini index 2d2c6f3974..b2e0e86c2f 100644 --- a/src/replica/test/config-test.ini +++ b/src/replica/test/config-test.ini @@ -34,7 +34,7 @@ type = replica run = true count = 1 ports = 54321 -pools = THREAD_POOL_DEFAULT,THREAD_POOL_REPLICATION_LONG,THREAD_POOL_REPLICATION,THREAD_POOL_PLOG,THREAD_POOL_BLOCK_SERVICE +pools = THREAD_POOL_DEFAULT,THREAD_POOL_REPLICATION_LONG,THREAD_POOL_LOCAL_APP,THREAD_POOL_REPLICATION,THREAD_POOL_PLOG,THREAD_POOL_BLOCK_SERVICE [core] ;tool = simulator @@ -76,6 +76,12 @@ partitioned = true worker_priority = THREAD_xPRIORITY_NORMAL worker_count = 3 +[threadpool.THREAD_POOL_LOCAL_APP] +name = local_app +partitioned = false +worker_priority = THREAD_xPRIORITY_NORMAL +worker_count = 4 + [threadpool.THREAD_POOL_REPLICATION_LONG] name = replica_long diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp new file mode 100644 index 0000000000..7521fc2ec9 --- /dev/null +++ b/src/replica/test/load_replicas_test.cpp @@ -0,0 +1,137 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include + +#include "common/fs_manager.h" +#include "common/gpid.h" +#include "gtest/gtest.h" +#include "replica/replica_stub.h" +#include "replica/test/mock_utils.h" +#include "utils/filesystem.h" + +namespace dsn { +namespace replication { + +struct load_replicas_case +{ + std::map dirs_by_tag; + std::map> replicas_by_tag; +}; + +class LoadReplicasTest : public replica_stub, public testing::TestWithParam +{ +public: + LoadReplicasTest() {} + + ~LoadReplicasTest() override = default; + + void initialize(const std::map &dirs_by_tag, + const std::map> &replicas_by_tag) + { + // + std::vector dirs; + std::vector tags; + for (const auto &[tag, dir] : dirs_by_tag) { + dirs.push_back(dir); + tags.push_back(tag); + } + + for (const auto &[tag, reps] : replicas_by_tag) { + for (const auto &pid : reps) { + ASSERT_TRUE(_expected_pids.insert(pid).second); + } + } + + // + _fs_manager.initialize(dirs, tags); + + for (const auto &dn : _fs_manager.get_dir_nodes()) { + for (const auto &pid : replicas_by_tag.at(dn->tag)) { + _fs_manager.specify_dir_for_new_replica_for_test(dn.get(), "pegasus", pid); + } + } + } + + void test_load_replicas() + { + replicas reps; + load_replicas(reps); + ASSERT_EQ(_loaded_replicas, reps); + + std::set actual_pids; + for (const auto &[pid, _] : reps) { + ASSERT_TRUE(actual_pids.insert(pid).second); + } + ASSERT_EQ(_expected_pids, actual_pids); + } + +private: + void load_replica_for_test(dir_node *dn, const char *dir, replica_ptr &rep) + { + ASSERT_TRUE(utils::filesystem::directory_exists(dir)); + + const auto &dir_name = get_replica_dir_name(dir); + + gpid pid; + std::string app_type; + ASSERT_TRUE(parse_replica_dir_name(dir_name, pid, app_type)); + ASSERT_STREQ("pegasus", app_type.c_str()); + + // Check full dir. + ASSERT_EQ(dn->replica_dir("pegasus", pid), dir); + + std::lock_guard guard(_mtx); + + ASSERT_TRUE(_loaded_replicas.find(pid) == _loaded_replicas.end()); + + app_info ai; + ai.app_type = "pegasus"; + rep = new replica(this, pid, ai, dn, false); + rep->_app = std::make_unique(rep); + _loaded_replicas[pid] = rep; + } + + replica_ptr load_replica(dir_node *dn, const char *dir) override + { + replica_ptr rep; + load_replica_for_test(dn, dir, rep); + return rep; + } + + std::set _expected_pids; + + mutable std::mutex _mtx; + replicas _loaded_replicas; +}; + +TEST_P(LoadReplicasTest, LoadReplicas) +{ + const auto &load_case = GetParam(); + initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); + test_load_replicas(); +} + +const std::vector load_replicas_tests = { + {{{"data0", "disk0"}}, {{"data0", {{1, 2}, {2, 5}}}}}}; + +INSTANTIATE_TEST_SUITE_P(ReplicaStubTest, LoadReplicasTest, testing::ValuesIn(load_replicas_tests)); + +} // namespace replication +} // namespace dsn diff --git a/src/utils/autoref_ptr.h b/src/utils/autoref_ptr.h index 501698f852..c3ba9d9595 100644 --- a/src/utils/autoref_ptr.h +++ b/src/utils/autoref_ptr.h @@ -160,6 +160,15 @@ class ref_ptr T *operator->() const { return _obj; } + bool operator==(const ref_ptr &r) const { return _obj == r._obj; } + + template ::value>::type> + bool operator==(const ref_ptr &r) const + { + return _obj == r._obj; + } + bool operator==(T *r) const { return _obj == r; } bool operator!=(T *r) const { return _obj != r; } From 37a21cd54bc90ebeab9819313dfcb7ff54100044 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Thu, 25 Jul 2024 14:48:31 +0800 Subject: [PATCH 10/43] add tests --- src/replica/test/load_replicas_test.cpp | 69 +++++++++++++++++++++++-- 1 file changed, 66 insertions(+), 3 deletions(-) diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 7521fc2ec9..a0cbbd218f 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -15,15 +15,26 @@ // specific language governing permissions and limitations // under the License. +#include #include +#include +#include +#include #include +#include +#include +#include #include #include "common/fs_manager.h" #include "common/gpid.h" +#include "dsn.layer2_types.h" #include "gtest/gtest.h" +#include "replica/replica.h" #include "replica/replica_stub.h" +#include "replica/replication_app_base.h" #include "replica/test/mock_utils.h" +#include "utils/autoref_ptr.h" #include "utils/filesystem.h" namespace dsn { @@ -82,6 +93,13 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParamfull_dir)); + } + } + private: void load_replica_for_test(dir_node *dn, const char *dir, replica_ptr &rep) { @@ -126,12 +144,57 @@ TEST_P(LoadReplicasTest, LoadReplicas) const auto &load_case = GetParam(); initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); test_load_replicas(); + remove_disk_dirs(); +} + +load_replicas_case generate_load_replicas_case(const std::vector &replicas_per_disk) +{ + static const int32_t kNumPartitions = 8; + + std::map dirs_by_tag; + for (size_t disk_index = 0; disk_index < replicas_per_disk.size(); ++disk_index) { + dirs_by_tag.emplace(fmt::format("data{}", disk_index), fmt::format("disk{}", disk_index)); + } + + int32_t app_id = 1; + int32_t partition_id = 0; + std::map> replicas_by_tag; + for (size_t disk_index = 0; disk_index < replicas_per_disk.size(); ++disk_index) { + std::vector pids; + pids.reserve(replicas_per_disk[disk_index]); + + for (size_t replica_index = 0; replica_index < replicas_per_disk[disk_index]; + ++replica_index) { + pids.emplace_back(app_id, partition_id); + if (++partition_id >= kNumPartitions) { + ++app_id; + partition_id = 0; + } + } + + replicas_by_tag.emplace(fmt::format("data{}", disk_index), pids); + } + + return {dirs_by_tag, replicas_by_tag}; } -const std::vector load_replicas_tests = { - {{{"data0", "disk0"}}, {{"data0", {{1, 2}, {2, 5}}}}}}; +std::vector generate_load_replicas_cases() +{ + return std::vector({ + // at least 1 disk dir + generate_load_replicas_case({0}), + generate_load_replicas_case({0, 0}), + generate_load_replicas_case({1}), + generate_load_replicas_case({1, 0}), + generate_load_replicas_case({2}), + generate_load_replicas_case({1, 0, 2}), + generate_load_replicas_case({50, 30, 100, 200, 80}), + }); +} -INSTANTIATE_TEST_SUITE_P(ReplicaStubTest, LoadReplicasTest, testing::ValuesIn(load_replicas_tests)); +INSTANTIATE_TEST_SUITE_P(ReplicaStubTest, + LoadReplicasTest, + testing::ValuesIn(generate_load_replicas_cases())); } // namespace replication } // namespace dsn From 293971b6bb3eb159af0bc271ad872c0714698122 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 26 Jul 2024 16:24:13 +0800 Subject: [PATCH 11/43] add tests --- src/replica/replica_stub.cpp | 11 ++++ src/replica/test/load_replicas_test.cpp | 71 +++++++++++++++++-------- src/test_util/test_util.h | 9 ++-- 3 files changed, 65 insertions(+), 26 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 0b9b21d4fd..4d61142611 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -71,6 +71,9 @@ #include "security/access_controller.h" #include "split/replica_split_manager.h" #include "task/async_calls.h" +#include "task/task.h" +#include "task/task_engine.h" +#include "task/task_worker.h" #include "utils/command_manager.h" #include "utils/env.h" #include "utils/errors.h" @@ -82,6 +85,7 @@ #include "utils/rand.h" #include "utils/strings.h" #include "utils/synchronize.h" +#include "utils/threadpool_spec.h" #include "utils/time_utils.h" #ifdef DSN_ENABLE_GPERF #include @@ -480,6 +484,13 @@ void replica_stub::load_replica(dir_node *dn, { LOG_INFO("process dir {}", dir); + const auto *const worker = task::get_current_worker2(); + if (worker != nullptr) { + CHECK(!(worker->pool()->spec().partitioned), + "The thread pool for loading replicas must not be partitioned since load balancing " + "is required among multiple threads"); + } + auto rep = load_replica(dn, dir.c_str()); if (rep == nullptr) { return; diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index a0cbbd218f..b835ad68f2 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -16,6 +16,8 @@ // under the License. #include +#include +#include #include #include #include @@ -34,6 +36,8 @@ #include "replica/replica_stub.h" #include "replica/replication_app_base.h" #include "replica/test/mock_utils.h" +#include "runtime/task/task.h" +#include "test_util/test_util.h" #include "utils/autoref_ptr.h" #include "utils/filesystem.h" @@ -49,7 +53,9 @@ struct load_replicas_case class LoadReplicasTest : public replica_stub, public testing::TestWithParam { public: - LoadReplicasTest() {} + static const int32_t kAppId; + + LoadReplicasTest() = default; ~LoadReplicasTest() override = default; @@ -80,8 +86,11 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParamallow_inline); + dsn::task_spec::get(LPC_REPLICATION_INIT_LOAD)->allow_inline = test_load_order; + replicas reps; load_replicas(reps); ASSERT_EQ(_loaded_replicas, reps); @@ -100,6 +109,8 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParamspec().code); + if (task::get_current_task()->spec().allow_inline) { + ASSERT_EQ(gpid(kAppId, _partition_id++), pid); + } + // Check full dir. ASSERT_EQ(dn->replica_dir("pegasus", pid), dir); - std::lock_guard guard(_mtx); - - ASSERT_TRUE(_loaded_replicas.find(pid) == _loaded_replicas.end()); - app_info ai; ai.app_type = "pegasus"; rep = new replica(this, pid, ai, dn, false); rep->_app = std::make_unique(rep); + + std::lock_guard guard(_mtx); + + ASSERT_TRUE(_loaded_replicas.find(pid) == _loaded_replicas.end()); + _loaded_replicas[pid] = rep; } @@ -134,45 +152,54 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParam _expected_pids; + int32_t _partition_id{0}; mutable std::mutex _mtx; replicas _loaded_replicas; }; +const int32_t LoadReplicasTest::kAppId = 1; + TEST_P(LoadReplicasTest, LoadReplicas) { const auto &load_case = GetParam(); initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(); - remove_disk_dirs(); + test_load_replicas(false); } -load_replicas_case generate_load_replicas_case(const std::vector &replicas_per_disk) +TEST_P(LoadReplicasTest, LoadOrder) { - static const int32_t kNumPartitions = 8; + const auto &load_case = GetParam(); + initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); + test_load_replicas(true); +} +load_replicas_case generate_load_replicas_case(const std::vector &replicas_per_disk) +{ std::map dirs_by_tag; for (size_t disk_index = 0; disk_index < replicas_per_disk.size(); ++disk_index) { dirs_by_tag.emplace(fmt::format("data{}", disk_index), fmt::format("disk{}", disk_index)); } - int32_t app_id = 1; int32_t partition_id = 0; std::map> replicas_by_tag; - for (size_t disk_index = 0; disk_index < replicas_per_disk.size(); ++disk_index) { - std::vector pids; - pids.reserve(replicas_per_disk[disk_index]); - - for (size_t replica_index = 0; replica_index < replicas_per_disk[disk_index]; - ++replica_index) { - pids.emplace_back(app_id, partition_id); - if (++partition_id >= kNumPartitions) { - ++app_id; - partition_id = 0; + + while (true) { + size_t finished_disks = 0; + + for (size_t disk_index = 0; disk_index < replicas_per_disk.size(); ++disk_index) { + auto &replica_list = replicas_by_tag[fmt::format("data{}", disk_index)]; + if (replica_list.size() >= replicas_per_disk[disk_index]) { + ++finished_disks; + continue; } + + replica_list.emplace_back(LoadReplicasTest::kAppId, partition_id++); } - replicas_by_tag.emplace(fmt::format("data{}", disk_index), pids); + if (finished_disks >= replicas_per_disk.size()) { + break; + } } return {dirs_by_tag, replicas_by_tag}; diff --git a/src/test_util/test_util.h b/src/test_util/test_util.h index 2e2b34bb0d..549d8fd948 100644 --- a/src/test_util/test_util.h +++ b/src/test_util/test_util.h @@ -44,11 +44,12 @@ class file_meta; } // namespace replication } // namespace dsn +#define PRESERVE_VAR(name, expr) \ + const auto PRESERVED_##name = expr; \ + auto PRESERVED_##name##_cleanup = dsn::defer([PRESERVED_##name]() { expr = PRESERVED_##name; }) + // Save the current value of a flag and restore it at the end of the function. -#define PRESERVE_FLAG(name) \ - const auto PRESERVED_FLAGS_##name = FLAGS_##name; \ - auto PRESERVED_FLAGS_##name##_cleanup = \ - dsn::defer([PRESERVED_FLAGS_##name]() { FLAGS_##name = PRESERVED_FLAGS_##name; }) +#define PRESERVE_FLAG(name) PRESERVE_VAR(FLAGS_##name, FLAGS_##name) namespace pegasus { From 4efcd564ae3544ac0d6bcd7f9a41326b359d4099 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 26 Jul 2024 19:12:27 +0800 Subject: [PATCH 12/43] fix tests --- src/replica/replica_stub.cpp | 8 +- src/replica/test/load_replicas_test.cpp | 102 +++++++++++++++++------- 2 files changed, 81 insertions(+), 29 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 4d61142611..dc212485de 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -270,6 +270,11 @@ DSN_DEFINE_uint64(replication, 256, "The max number of replicas that are allowed to be loaded simultaneously " "for each disk dir."); +DSN_TAG_VARIABLE(max_replicas_on_load_for_each_disk, FT_MUTABLE); +DSN_DEFINE_validator(max_replicas_on_load_for_each_disk, + [](uint64_t max_replicas_on_load_for_each_disk) -> bool { + return max_replicas_on_load_for_each_disk > 0; + }); DSN_DEFINE_uint64(replication, load_replica_max_wait_time_ms, @@ -534,7 +539,8 @@ void replica_stub::load_replicas(replicas &reps) } auto &load_disk_queue = load_disk_queues[disk_index]; - if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { + if (!load_disk_queue.empty() && + load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { if (load_disk_queue.front()->wait(FLAGS_load_replica_max_wait_time_ms)) { load_disk_queue.pop(); } diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index b835ad68f2..a40c605c9d 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -30,6 +30,7 @@ #include "common/fs_manager.h" #include "common/gpid.h" +#include "common/replication.codes.h" #include "dsn.layer2_types.h" #include "gtest/gtest.h" #include "replica/replica.h" @@ -37,10 +38,14 @@ #include "replica/replication_app_base.h" #include "replica/test/mock_utils.h" #include "runtime/task/task.h" +#include "runtime/task/task_code.h" +#include "runtime/task/task_spec.h" #include "test_util/test_util.h" #include "utils/autoref_ptr.h" #include "utils/filesystem.h" +DSN_DECLARE_uint64(max_replicas_on_load_for_each_disk); + namespace dsn { namespace replication { @@ -53,8 +58,6 @@ struct load_replicas_case class LoadReplicasTest : public replica_stub, public testing::TestWithParam { public: - static const int32_t kAppId; - LoadReplicasTest() = default; ~LoadReplicasTest() override = default; @@ -72,34 +75,47 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParamtag)) { _fs_manager.specify_dir_for_new_replica_for_test(dn.get(), "pegasus", pid); } + + _disk_tags_for_order.push_back(dn->tag); + _disk_dirs_for_order.push_back(dn->full_dir); + _disk_replicas_for_order.push_back(replicas_by_tag.at(dn->tag).size()); } + + ASSERT_EQ(_disk_tags_for_order.size(), _disk_dirs_for_order.size()); } - void test_load_replicas(bool test_load_order) + void test_load_replicas(bool test_load_order, uint64_t max_replicas_on_load_for_each_disk) { PRESERVE_VAR(allow_inline, dsn::task_spec::get(LPC_REPLICATION_INIT_LOAD)->allow_inline); dsn::task_spec::get(LPC_REPLICATION_INIT_LOAD)->allow_inline = test_load_order; - replicas reps; - load_replicas(reps); - ASSERT_EQ(_loaded_replicas, reps); + PRESERVE_FLAG(max_replicas_on_load_for_each_disk); + FLAGS_max_replicas_on_load_for_each_disk = max_replicas_on_load_for_each_disk; + + replicas actual_loaded_replicas; + load_replicas(actual_loaded_replicas); + ASSERT_EQ(_expected_loaded_replicas, actual_loaded_replicas); - std::set actual_pids; - for (const auto &[pid, _] : reps) { - ASSERT_TRUE(actual_pids.insert(pid).second); + std::set actual_loaded_replica_pids; + for (const auto &[pid, _] : actual_loaded_replicas) { + ASSERT_TRUE(actual_loaded_replica_pids.insert(pid).second); } - ASSERT_EQ(_expected_pids, actual_pids); + ASSERT_EQ(_expected_loaded_replica_pids, actual_loaded_replica_pids); } void remove_disk_dirs() @@ -123,10 +139,24 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParamspec().code); if (task::get_current_task()->spec().allow_inline) { - ASSERT_EQ(gpid(kAppId, _partition_id++), pid); + size_t finished_disks = 0; + while (finished_disks < _disk_tags_for_order.size() && + _disk_loaded_replicas_for_order[_disk_index_for_order] >= + _disk_replicas_for_order[_disk_index_for_order]) { + ++finished_disks; + _disk_index_for_order = (_disk_index_for_order + 1) % _disk_tags_for_order.size(); + } + + ASSERT_GT(_disk_tags_for_order.size(), finished_disks); + + ASSERT_EQ(_disk_tags_for_order[_disk_index_for_order], dn->tag); + ASSERT_EQ(_disk_dirs_for_order[_disk_index_for_order], dn->full_dir); + + ++_disk_loaded_replicas_for_order[_disk_index_for_order]; + _disk_index_for_order = (_disk_index_for_order + 1) % _disk_tags_for_order.size(); } // Check full dir. @@ -139,9 +169,9 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParam guard(_mtx); - ASSERT_TRUE(_loaded_replicas.find(pid) == _loaded_replicas.end()); + ASSERT_TRUE(_expected_loaded_replicas.find(pid) == _expected_loaded_replicas.end()); - _loaded_replicas[pid] = rep; + _expected_loaded_replicas[pid] = rep; } replica_ptr load_replica(dir_node *dn, const char *dir) override @@ -151,53 +181,69 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParam _expected_pids; - int32_t _partition_id{0}; + std::set _expected_loaded_replica_pids; + + size_t _disk_index_for_order{0}; + std::vector _disk_tags_for_order; + std::vector _disk_dirs_for_order; + std::vector _disk_replicas_for_order; + std::vector _disk_loaded_replicas_for_order; mutable std::mutex _mtx; - replicas _loaded_replicas; + replicas _expected_loaded_replicas; }; -const int32_t LoadReplicasTest::kAppId = 1; - TEST_P(LoadReplicasTest, LoadReplicas) { const auto &load_case = GetParam(); initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(false); + test_load_replicas(false, FLAGS_max_replicas_on_load_for_each_disk); } TEST_P(LoadReplicasTest, LoadOrder) { const auto &load_case = GetParam(); initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(true); + test_load_replicas(true, FLAGS_max_replicas_on_load_for_each_disk); } -load_replicas_case generate_load_replicas_case(const std::vector &replicas_per_disk) +TEST_P(LoadReplicasTest, LoadThrottling) +{ + const auto &load_case = GetParam(); + initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); + test_load_replicas(false, 1); +} + +load_replicas_case generate_load_replicas_case(const std::vector &disk_replicas) { std::map dirs_by_tag; - for (size_t disk_index = 0; disk_index < replicas_per_disk.size(); ++disk_index) { + for (size_t disk_index = 0; disk_index < disk_replicas.size(); ++disk_index) { dirs_by_tag.emplace(fmt::format("data{}", disk_index), fmt::format("disk{}", disk_index)); } + static int32_t kNumPartitions = 8; int32_t partition_id = 0; + int32_t app_id = 1; std::map> replicas_by_tag; while (true) { size_t finished_disks = 0; - for (size_t disk_index = 0; disk_index < replicas_per_disk.size(); ++disk_index) { + for (size_t disk_index = 0; disk_index < disk_replicas.size(); ++disk_index) { auto &replica_list = replicas_by_tag[fmt::format("data{}", disk_index)]; - if (replica_list.size() >= replicas_per_disk[disk_index]) { + if (replica_list.size() >= disk_replicas[disk_index]) { ++finished_disks; continue; } - replica_list.emplace_back(LoadReplicasTest::kAppId, partition_id++); + replica_list.emplace_back(app_id, partition_id); + if (++partition_id >= kNumPartitions) { + partition_id = 0; + ++app_id; + } } - if (finished_disks >= replicas_per_disk.size()) { + if (finished_disks >= disk_replicas.size()) { break; } } From 461ac0c70db10720642e81d83fa3cec26366b055 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Mon, 29 Jul 2024 11:56:01 +0800 Subject: [PATCH 13/43] add tests --- src/replica/test/load_replicas_test.cpp | 28 ++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index a40c605c9d..07e3fb00e2 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -43,6 +43,7 @@ #include "test_util/test_util.h" #include "utils/autoref_ptr.h" #include "utils/filesystem.h" +#include "utils/flags.h" DSN_DECLARE_uint64(max_replicas_on_load_for_each_disk); @@ -197,21 +198,21 @@ TEST_P(LoadReplicasTest, LoadReplicas) { const auto &load_case = GetParam(); initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(false, FLAGS_max_replicas_on_load_for_each_disk); + test_load_replicas(false, 256); } TEST_P(LoadReplicasTest, LoadOrder) { const auto &load_case = GetParam(); initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(true, FLAGS_max_replicas_on_load_for_each_disk); + test_load_replicas(true, 256); } TEST_P(LoadReplicasTest, LoadThrottling) { const auto &load_case = GetParam(); initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(false, 1); + test_load_replicas(false, 5); } load_replicas_case generate_load_replicas_case(const std::vector &disk_replicas) @@ -253,15 +254,32 @@ load_replicas_case generate_load_replicas_case(const std::vector &disk_r std::vector generate_load_replicas_cases() { + // At least 1 disk should be included (otherwise it would lead to core dump), thus do + // not generate the empty case (i.e. {}). return std::vector({ - // at least 1 disk dir + // There is only one disk which has none of replica. generate_load_replicas_case({0}), + // There are two disks both of which have none of replica. generate_load_replicas_case({0, 0}), + // There is only one disk which has one replica. generate_load_replicas_case({1}), + // There are two disks one of which has one replica, and another has none. generate_load_replicas_case({1, 0}), + generate_load_replicas_case({0, 1}), + // There is only one disk which has two replicas. generate_load_replicas_case({2}), + // There are two disks one of which has two replicas, and another has none. + generate_load_replicas_case({2, 0}), + generate_load_replicas_case({0, 2}), + // There are at least three disks. generate_load_replicas_case({1, 0, 2}), - generate_load_replicas_case({50, 30, 100, 200, 80}), + generate_load_replicas_case({8, 25, 16}), + generate_load_replicas_case({17, 96, 56, 127}), + generate_load_replicas_case({22, 38, 0, 16}), + generate_load_replicas_case({82, 75, 36, 118, 65}), + // There are many replicas for some disks. + generate_load_replicas_case({156, 367, 309, 58, 404, 298, 512, 82}), + generate_load_replicas_case({167, 28, 898, 516, 389, 422, 682, 265, 596}), }); } From dd875022bb85072ca9cd5fff05b87b19efa31cbc Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Mon, 29 Jul 2024 18:45:13 +0800 Subject: [PATCH 14/43] add comments --- src/replica/replica_stub.cpp | 22 +++++++++++-- src/replica/replica_stub.h | 41 +++++++++++++++++-------- src/replica/test/load_replicas_test.cpp | 15 ++++----- 3 files changed, 56 insertions(+), 22 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index dc212485de..99fb3a23e7 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -467,8 +467,8 @@ replica_stub::disk_dirs replica_stub::get_all_disk_dirs() const { disk_dirs disks; for (const auto &dn : _fs_manager.get_dir_nodes()) { - // Skip dir node with IO error. if (dsn_unlikely(dn->status == disk_status::IO_ERROR)) { + // Skip disks with IO errors. continue; } @@ -530,10 +530,13 @@ void replica_stub::load_replicas(replicas &reps) while (true) { size_t finished_disks = 0; + // For each round, start loading one replica for each disk in case there are too many + // replicas in a disk, except that all of the replicas of this disk are being loaded. for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { auto &dir_index = dir_indexes[disk_index]; const auto &dirs = disks[disk_index].second; if (dir_index >= dirs.size()) { + // All of the replicas for the disk `disks[disk_index]` have begun to be loaded. ++finished_disks; continue; } @@ -541,10 +544,20 @@ void replica_stub::load_replicas(replicas &reps) auto &load_disk_queue = load_disk_queues[disk_index]; if (!load_disk_queue.empty() && load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { + // Loading replicas should be throttled in case that disk IO is saturated. if (load_disk_queue.front()->wait(FLAGS_load_replica_max_wait_time_ms)) { load_disk_queue.pop(); + } else { + // There might be too many replicas that are being loaded which lead to + // slow disk IO. + continue; + } + + // Continue to load a replica since we are within the limit now. + if (dsn_unlikely(load_disk_queue.size() >= + FLAGS_max_replicas_on_load_for_each_disk)) { + continue; } - continue; } const auto &dir = dirs[dir_index++]; @@ -580,10 +593,12 @@ void replica_stub::load_replicas(replicas &reps) } if (finished_disks >= disks.size()) { + // All replicas of all disks have begun to be loaded. break; } } + // All loading tasks have been in the queue. Just wait all tasks to be finished. for (auto &load_disk_queue : load_disk_queues) { while (!load_disk_queue.empty()) { CHECK_TRUE(load_disk_queue.front()->wait()); @@ -678,6 +693,7 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f LOG_INFO("start to load replicas"); replicas reps; + utils::chronograph chrono; load_replicas(reps); @@ -2085,7 +2101,7 @@ replica *replica_stub::new_replica(gpid gpid, /* static */ bool replica_stub::parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type) { - int32_t app_id, partition_id; + int32_t app_id = 0, partition_id = 0; char app_type_buf[128] = {0}; if (3 != sscanf(dir_name.c_str(), "%d.%d.%s", &app_id, &partition_id, app_type_buf)) { return false; diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index a54641b6a7..ded7b90ab8 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -343,17 +343,6 @@ class replica_stub : public serverlet, public ref_counter RL_closed }; - using disk_dirs = std::vector>>; - disk_dirs get_all_disk_dirs() const; - - static std::string get_replica_dir_name(const std::string &dir); - static bool - parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type); - - void - load_replica(dir_node *dn, const std::string &dir, utils::ex_lock &reps_lock, replicas &reps); - void load_replicas(replicas &reps); - void initialize_start(); void query_configuration_by_node(); void on_meta_server_disconnected_scatter(replica_stub_ptr this_, gpid id); @@ -377,8 +366,36 @@ class replica_stub : public serverlet, public ref_counter bool restore_if_necessary, bool is_duplication_follower, const std::string &parent_dir = ""); - // Load an existing replica which is located in 'dn' with 'dir' directory. + + using disk_dirs = std::vector>>; + + // Get the absolute dirs of all replicas for all disks. + disk_dirs get_all_disk_dirs() const; + + // Get the dir name for a replica from a potentially longer path (both absolute and + // relative paths are possible). + static std::string get_replica_dir_name(const std::string &dir); + + // Parse app id, partition id and app type from the dir name of a replica. + static bool + parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type); + + // Load an existing replica which is located in `dn` with `dir`. Usually each different + // `dn` represents a unique disk. `dir` is the absolute path of the directory for a + // replica. virtual replica_ptr load_replica(dir_node *dn, const char *dir); + + // The same as the above `load_replica` function, except that this function is to load + // each replica to `reps` with protection from `reps_lock`. + void + load_replica(dir_node *dn, const std::string &dir, utils::ex_lock &reps_lock, replicas &reps); + + // Load all replicas synchronously from all disks to `reps`. This function would ensure + // that data on each disk is loaded more evenly, rather than that a disk would begin to + // be loaded only after another has been finished, in case that there are too many replicas + // on a disk and other disks cannot start loading until this disk is finished. + void load_replicas(replicas &reps); + // Clean up the memory state and on disk data if creating replica failed. void clear_on_failure(replica *rep); task_ptr begin_close_replica(replica_ptr r); diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 07e3fb00e2..0ddc8ef28b 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -66,7 +66,7 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParam &dirs_by_tag, const std::map> &replicas_by_tag) { - // + // Get dirs and tags to initialize fs_manager. std::vector dirs; std::vector tags; for (const auto &[tag, dir] : dirs_by_tag) { @@ -74,13 +74,14 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParamspec().code); if (task::get_current_task()->spec().allow_inline) { size_t finished_disks = 0; - while (finished_disks < _disk_tags_for_order.size() && - _disk_loaded_replicas_for_order[_disk_index_for_order] >= - _disk_replicas_for_order[_disk_index_for_order]) { + while (_disk_loaded_replicas_for_order[_disk_index_for_order] >= + _disk_replicas_for_order[_disk_index_for_order]) { + // ++finished_disks; + ASSERT_GT(_disk_tags_for_order.size(), finished_disks); + _disk_index_for_order = (_disk_index_for_order + 1) % _disk_tags_for_order.size(); } - ASSERT_GT(_disk_tags_for_order.size(), finished_disks); - ASSERT_EQ(_disk_tags_for_order[_disk_index_for_order], dn->tag); ASSERT_EQ(_disk_dirs_for_order[_disk_index_for_order], dn->full_dir); From 544c580c9bf443a0dd20580771710d5dc9668b93 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Tue, 30 Jul 2024 15:13:50 +0800 Subject: [PATCH 15/43] refactor --- src/replica/replica_stub.cpp | 3 +- src/replica/test/load_replicas_test.cpp | 22 +++++-- src/utils/simple_concurrent_queue.h | 88 ------------------------- 3 files changed, 20 insertions(+), 93 deletions(-) delete mode 100644 src/utils/simple_concurrent_queue.h diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 99fb3a23e7..c971faf2de 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -536,7 +536,8 @@ void replica_stub::load_replicas(replicas &reps) auto &dir_index = dir_indexes[disk_index]; const auto &dirs = disks[disk_index].second; if (dir_index >= dirs.size()) { - // All of the replicas for the disk `disks[disk_index]` have begun to be loaded. + // All of the replicas for the disk `disks[disk_index]` have begun to be loaded, + // thus just skip. ++finished_disks; continue; } diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 0ddc8ef28b..f850b405ef 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -141,27 +141,38 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParamspec().code); + if (task::get_current_task()->spec().allow_inline) { + // Once the task is `allow_inline`, it would be executed in place immediately rather + // than pushed into the queue. Thus we could test the expected order in which the + // tasks are pushed into the queue. size_t finished_disks = 0; while (_disk_loaded_replicas_for_order[_disk_index_for_order] >= _disk_replicas_for_order[_disk_index_for_order]) { - // + // Since current task has not been executed, it is not possible that all disks + // are finished. ++finished_disks; ASSERT_GT(_disk_tags_for_order.size(), finished_disks); + // Skip to next disk since all of the replicas of this disk have been loaded. _disk_index_for_order = (_disk_index_for_order + 1) % _disk_tags_for_order.size(); } + // Only check if the processed order of the disk the replica belongs to, rather than + // the order of the replica itself, for the reason that the order of the dirs returned + // by the underlying call varies with different systems. ASSERT_EQ(_disk_tags_for_order[_disk_index_for_order], dn->tag); ASSERT_EQ(_disk_dirs_for_order[_disk_index_for_order], dn->full_dir); + // Current replica has been loaded, move forward to the next replica of this disk. ++_disk_loaded_replicas_for_order[_disk_index_for_order]; + + // Turn to next disks if some of them still have some replicas that are not loaded. _disk_index_for_order = (_disk_index_for_order + 1) % _disk_tags_for_order.size(); } - // Check full dir. + // Check the absolute dir of this replica. ASSERT_EQ(dn->replica_dir("pegasus", pid), dir); app_info ai; @@ -176,6 +187,7 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParam _expected_loaded_replica_pids; + // Only for testing the order of the loading tasks. size_t _disk_index_for_order{0}; std::vector _disk_tags_for_order; std::vector _disk_dirs_for_order; @@ -223,7 +236,7 @@ load_replicas_case generate_load_replicas_case(const std::vector &disk_r dirs_by_tag.emplace(fmt::format("data{}", disk_index), fmt::format("disk{}", disk_index)); } - static int32_t kNumPartitions = 8; + static const int32_t kNumPartitions = 8; int32_t partition_id = 0; int32_t app_id = 1; std::map> replicas_by_tag; @@ -278,6 +291,7 @@ std::vector generate_load_replicas_cases() generate_load_replicas_case({17, 96, 56, 127}), generate_load_replicas_case({22, 38, 0, 16}), generate_load_replicas_case({82, 75, 36, 118, 65}), + generate_load_replicas_case({0, 92, 17, 68, 25}), // There are many replicas for some disks. generate_load_replicas_case({156, 367, 309, 58, 404, 298, 512, 82}), generate_load_replicas_case({167, 28, 898, 516, 389, 422, 682, 265, 596}), diff --git a/src/utils/simple_concurrent_queue.h b/src/utils/simple_concurrent_queue.h deleted file mode 100644 index 835a9add8b..0000000000 --- a/src/utils/simple_concurrent_queue.h +++ /dev/null @@ -1,88 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include -#include -#include - -#include "utils/ports.h" - -namespace dsn { - -template -class simple_concurrent_queue -{ -public: - using value_type = T; - - simple_concurrent_queue() = default; - - ~simple_concurrent_queue() = default; - - void push(value_type &&value) - { - { - std::lock_guard lock(_mtx); - _queue.push(std::forward(value)); - } - - _cond_var.notify_one(); - } - - void pop(value_type &value) - { - std::unique_lock lock(_mtx); - _cond_var.wait(lock, [this] { return !_queue.empty(); }); - - value = _queue.front(); - _queue.pop(); - } - - bool pop(uint64_t timeout_ns, value_type &value) - { - std::unique_lock lock(_mtx); - const auto status = _cond_var.wait_for( - lock, std::chrono::nanoseconds(timeout_ns), [this] { return !_queue.empty(); }); - - if (status == std::cv_status::timeout) { - return false; - } - - value = _queue.front(); - _queue.pop(); - return true; - } - - size_t size() const - { - std::lock_guard lock(_mtx); - return _queue.size(); - } - -private: - std::queue _queue; - mutable std::mutex _mtx; - mutable std::condition_variable _cond_var; - - DISALLOW_COPY_AND_ASSIGN(simple_concurrent_queue); -}; - -} // namespace dsn From 9a0e79fadee27273e2d443323fcbe539d4f65155 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Tue, 30 Jul 2024 15:27:23 +0800 Subject: [PATCH 16/43] refactor --- src/replica/replica_stub.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index c971faf2de..be4c97491e 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -760,11 +760,11 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f std::chrono::seconds(FLAGS_disk_stat_interval_seconds)); } - // attach reps + // Attach `reps`. _replicas = std::move(reps); METRIC_VAR_INCREMENT_BY(total_replicas, _replicas.size()); - for (const auto &kv : _replicas) { - _fs_manager.add_replica(kv.first, kv.second->dir()); + for (const auto &[pid, rep] : _replicas) { + _fs_manager.add_replica(pid, rep->dir()); } _nfs = dsn::nfs_node::create(); From 93dbf5a46c1e2852c0931e353ee9d01c26e71fab Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Tue, 30 Jul 2024 17:11:29 +0800 Subject: [PATCH 17/43] refactor --- src/replica/replica_stub.cpp | 84 +++++++++++++++++++++++------------- 1 file changed, 53 insertions(+), 31 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index be4c97491e..5014b9de09 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -487,7 +487,7 @@ void replica_stub::load_replica(dir_node *dn, utils::ex_lock &reps_lock, replicas &reps) { - LOG_INFO("process dir {}", dir); + LOG_INFO("loading replica: tag={}, dir={}", dn->tag, dir); const auto *const worker = task::get_current_worker2(); if (worker != nullptr) { @@ -501,10 +501,11 @@ void replica_stub::load_replica(dir_node *dn, return; } - LOG_INFO("{}@{}: load replica '{}' success, = <{}, {}>, last_prepared_decree = {}", + LOG_INFO("{}@{}: load replica successfully, tag={}, dir={}, last_durable_decree={}, " + "last_committed_decree={}, last_prepared_decree={}", rep->get_gpid(), dsn_primary_host_port(), + dn->tag, dir, rep->last_durable_decree(), rep->last_committed_decree(), @@ -524,7 +525,7 @@ void replica_stub::load_replicas(replicas &reps) const auto &disks = get_all_disk_dirs(); std::vector dir_indexes(disks.size(), 0); - std::vector> load_disk_queues(disks.size()); + std::vector>> load_disk_queues(disks.size()); utils::ex_lock reps_lock; while (true) { @@ -533,8 +534,18 @@ void replica_stub::load_replicas(replicas &reps) // For each round, start loading one replica for each disk in case there are too many // replicas in a disk, except that all of the replicas of this disk are being loaded. for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - auto &dir_index = dir_indexes[disk_index]; + // Structured bindings can be captured by closures in g++, while not supported + // well by clang. Thus we do not use following statement to bind both variables + // until clang has been upgraded to version 16 which could support that well: + // + // const auto &[dn, dirs] = disks[disk_index]; + // + // For the docs of clang 16 please see: + // + // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support: const auto &dirs = disks[disk_index].second; + + auto &dir_index = dir_indexes[disk_index]; if (dir_index >= dirs.size()) { // All of the replicas for the disk `disks[disk_index]` have begun to be loaded, // thus just skip. @@ -542,15 +553,27 @@ void replica_stub::load_replicas(replicas &reps) continue; } + const auto &dn = disks[disk_index].first; auto &load_disk_queue = load_disk_queues[disk_index]; if (!load_disk_queue.empty() && load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { // Loading replicas should be throttled in case that disk IO is saturated. - if (load_disk_queue.front()->wait(FLAGS_load_replica_max_wait_time_ms)) { + if (load_disk_queue.front().second->wait(FLAGS_load_replica_max_wait_time_ms)) { load_disk_queue.pop(); } else { // There might be too many replicas that are being loaded which lead to // slow disk IO. + LOG_WARNING("after {} ms, loading dir({}) is still not finished, there are " + "{} replicas being loaded for disk(index={}, tag={}, path={}), " + "skip dir(index={}, path={}), turn to next disk", + FLAGS_load_replica_max_wait_time_ms, + load_disk_queue.front().first, + load_disk_queue.size(), + disk_index, + dn->tag, + dn->full_dir, + dir_index, + dirs[dir_index]); continue; } @@ -561,36 +584,35 @@ void replica_stub::load_replicas(replicas &reps) } } + LOG_DEBUG("ready to load dir(index={}, path={}) for disk(index={}, tag={}, path={})", + dir_index, + dirs[dir_index], + disk_index, + dn->tag, + dn->full_dir); + const auto &dir = dirs[dir_index++]; if (dsn::replication::is_data_dir_invalid(dir)) { LOG_WARNING("ignore dir {}", dir); continue; } - // Structured bindings can be captured by closures in g++, while not supported - // well by clang. Thus we do not use following statement to bind both variables - // until clang has been upgraded to version 16 which could support that well: - // - // const auto &[dn, dirs] = disks[disk_index]; - // - // For the docs of clang 16 please see: - // - // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support: - const auto &dn = disks[disk_index].first; - load_disk_queue.push(tasking::create_task( - // Ensure that the thread pool is non-partitioned. - LPC_REPLICATION_INIT_LOAD, - &_tracker, - std::bind(static_cast( - &replica_stub::load_replica), - this, - dn, - dir, - std::ref(reps_lock), - std::ref(reps)))); - - load_disk_queue.back()->enqueue(); + load_disk_queue.emplace( + dir, + tasking::create_task( + // Ensure that the thread pool is non-partitioned. + LPC_REPLICATION_INIT_LOAD, + &_tracker, + std::bind(static_cast( + &replica_stub::load_replica), + this, + dn, + dir, + std::ref(reps_lock), + std::ref(reps)))); + + load_disk_queue.back().second->enqueue(); } if (finished_disks >= disks.size()) { @@ -602,7 +624,7 @@ void replica_stub::load_replicas(replicas &reps) // All loading tasks have been in the queue. Just wait all tasks to be finished. for (auto &load_disk_queue : load_disk_queues) { while (!load_disk_queue.empty()) { - CHECK_TRUE(load_disk_queue.front()->wait()); + CHECK_TRUE(load_disk_queue.front().second->wait()); load_disk_queue.pop(); } } From d5a9044cbfc95b9312eb5c947804ece877d68238 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 30 Aug 2024 17:43:23 +0800 Subject: [PATCH 18/43] fix clang tidy --- .clang-tidy | 2 +- build_tools/clang_tidy.py | 2 + src/replica/replica.h | 2 +- src/replica/replica_stub.cpp | 75 ++++++++++++++++--------- src/replica/replica_stub.h | 20 ++++--- src/replica/test/load_replicas_test.cpp | 73 +++++++++++++----------- src/utils/time_utils.h | 4 +- 7 files changed, 108 insertions(+), 70 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index 914dee1198..95dd7616d7 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -20,7 +20,7 @@ CheckOptions: [] # Disable some checks that are not useful for us now. # They are sorted by names, and should be consistent to build_tools/clang_tidy.py. -Checks: 'abseil-*,boost-*,bugprone-*,cert-*,clang-analyzer-*,concurrency-*,cppcoreguidelines-*,darwin-*,fuchsia-*,google-*,hicpp-*,linuxkernel-*,llvm-*,misc-*,modernize-*,performance-*,portability-*,readability-*,-bugprone-easily-swappable-parameters,-bugprone-lambda-function-name,-bugprone-macro-parentheses,-cert-err58-cpp,-concurrency-mt-unsafe,-cppcoreguidelines-avoid-c-arrays,-cppcoreguidelines-avoid-magic-numbers,-cppcoreguidelines-avoid-non-const-global-variables,-cppcoreguidelines-macro-usage,-cppcoreguidelines-non-private-member-variables-in-classes,-cppcoreguidelines-owning-memory,-cppcoreguidelines-pro-bounds-array-to-pointer-decay,-cppcoreguidelines-pro-bounds-pointer-arithmetic,-cppcoreguidelines-pro-type-const-cast,-cppcoreguidelines-pro-type-union-access,-fuchsia-default-arguments-calls,-fuchsia-overloaded-operator,-fuchsia-statically-constructed-objects,-google-readability-avoid-underscore-in-googletest-name,-hicpp-avoid-c-arrays,-hicpp-named-parameter,-hicpp-no-array-decay,-llvm-include-order,-misc-definitions-in-headers,-misc-non-private-member-variables-in-classes,-misc-unused-parameters,-modernize-avoid-c-arrays,-modernize-replace-disallow-copy-and-assign-macro,-modernize-use-trailing-return-type,-performance-unnecessary-value-param,-readability-function-cognitive-complexity,-readability-identifier-length,-readability-magic-numbers,-readability-named-parameter' +Checks: 'abseil-*,boost-*,bugprone-*,cert-*,clang-analyzer-*,concurrency-*,cppcoreguidelines-*,darwin-*,fuchsia-*,google-*,hicpp-*,linuxkernel-*,llvm-*,misc-*,modernize-*,performance-*,portability-*,readability-*,-bugprone-easily-swappable-parameters,-bugprone-lambda-function-name,-bugprone-macro-parentheses,-cert-err58-cpp,-concurrency-mt-unsafe,-cppcoreguidelines-avoid-c-arrays,-cppcoreguidelines-avoid-magic-numbers,-cppcoreguidelines-avoid-non-const-global-variables,-cppcoreguidelines-macro-usage,-cppcoreguidelines-non-private-member-variables-in-classes,-cppcoreguidelines-owning-memory,-cppcoreguidelines-pro-bounds-array-to-pointer-decay,-cppcoreguidelines-pro-bounds-pointer-arithmetic,-cppcoreguidelines-pro-type-const-cast,-cppcoreguidelines-pro-type-union-access,-fuchsia-default-arguments-calls,-fuchsia-overloaded-operator,-fuchsia-statically-constructed-objects,-google-readability-avoid-underscore-in-googletest-name,-hicpp-avoid-c-arrays,-hicpp-named-parameter,-hicpp-no-array-decay,-llvm-include-order,-misc-definitions-in-headers,-misc-non-private-member-variables-in-classes,-misc-unused-parameters,-modernize-avoid-bind,-modernize-avoid-c-arrays,-modernize-replace-disallow-copy-and-assign-macro,-modernize-use-trailing-return-type,-performance-unnecessary-value-param,-readability-function-cognitive-complexity,-readability-identifier-length,-readability-magic-numbers,-readability-named-parameter,-readability-suspicious-call-argument' ExtraArgs: ExtraArgsBefore: [] FormatStyle: none diff --git a/build_tools/clang_tidy.py b/build_tools/clang_tidy.py index 09ea434b1d..07271b26a8 100755 --- a/build_tools/clang_tidy.py +++ b/build_tools/clang_tidy.py @@ -88,6 +88,7 @@ def tidy_on_path(path): "-misc-definitions-in-headers," "-misc-non-private-member-variables-in-classes," "-misc-unused-parameters," + "-modernize-avoid-bind," "-modernize-avoid-c-arrays," "-modernize-replace-disallow-copy-and-assign-macro," "-modernize-use-trailing-return-type," @@ -96,6 +97,7 @@ def tidy_on_path(path): "-readability-identifier-length," "-readability-magic-numbers," "-readability-named-parameter", + "-readability-suspicious-call-argument", "-extra-arg=-language=c++", "-extra-arg=-std=c++17", "-extra-arg=-Ithirdparty/output/include"] diff --git a/src/replica/replica.h b/src/replica/replica.h index a6bcfe4086..b312865c5e 100644 --- a/src/replica/replica.h +++ b/src/replica/replica.h @@ -610,7 +610,7 @@ class replica : public serverlet, public ref_counter, public replica_ba friend class replica_disk_test; friend class replica_disk_migrate_test; friend class open_replica_test; - friend class LoadReplicasTest; + friend class mock_load_replica; friend class replica_follower; friend class ::pegasus::server::pegasus_server_test_base; friend class ::pegasus::server::rocksdb_wrapper_test; diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 5014b9de09..3b1e2ff319 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -542,7 +542,7 @@ void replica_stub::load_replicas(replicas &reps) // // For the docs of clang 16 please see: // - // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support: + // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support. const auto &dirs = disks[disk_index].second; auto &dir_index = dir_indexes[disk_index]; @@ -558,7 +558,8 @@ void replica_stub::load_replicas(replicas &reps) if (!load_disk_queue.empty() && load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { // Loading replicas should be throttled in case that disk IO is saturated. - if (load_disk_queue.front().second->wait(FLAGS_load_replica_max_wait_time_ms)) { + if (load_disk_queue.front().second->wait( + static_cast(FLAGS_load_replica_max_wait_time_ms))) { load_disk_queue.pop(); } else { // There might be too many replicas that are being loaded which lead to @@ -755,8 +756,8 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f if (!is_log_complete) { LOG_ERROR("logs are not complete for some replicas, which means that shared log is " "truncated, mark all replicas as inactive"); - for (auto it = reps.begin(); it != reps.end(); ++it) { - it->second->set_inactive_state_transient(false); + for (auto &[_, rep] : reps) { + rep->set_inactive_state_transient(false); } } @@ -1507,27 +1508,28 @@ void replica_stub::on_node_query_reply(error_code err, resp.partitions.size(), resp.gc_replicas.size()); - replicas rs; + replicas reps; { zauto_read_lock rl(_replicas_lock); - rs = _replicas; + reps = _replicas; } - for (auto it = resp.partitions.begin(); it != resp.partitions.end(); ++it) { - rs.erase(it->config.pid); - tasking::enqueue(LPC_QUERY_NODE_CONFIGURATION_SCATTER, - &_tracker, - std::bind(&replica_stub::on_node_query_reply_scatter, this, this, *it), - it->config.pid.thread_hash()); + for (const auto &config_update : resp.partitions) { + reps.erase(config_update.config.pid); + tasking::enqueue( + LPC_QUERY_NODE_CONFIGURATION_SCATTER, + &_tracker, + std::bind(&replica_stub::on_node_query_reply_scatter, this, this, config_update), + config_update.config.pid.thread_hash()); } // For the replicas that do not exist on meta_servers. - for (auto it = rs.begin(); it != rs.end(); ++it) { + for (const auto &[pid, _] : reps) { tasking::enqueue( LPC_QUERY_NODE_CONFIGURATION_SCATTER2, &_tracker, - std::bind(&replica_stub::on_node_query_reply_scatter2, this, this, it->first), - it->first.thread_hash()); + std::bind(&replica_stub::on_node_query_reply_scatter2, this, this, pid), + pid.thread_hash()); } // handle the replicas which need to be gc @@ -1656,18 +1658,18 @@ void replica_stub::on_meta_server_disconnected() _state = NS_Disconnected; - replicas rs; + replicas reps; { zauto_read_lock rl(_replicas_lock); - rs = _replicas; + reps = _replicas; } - for (auto it = rs.begin(); it != rs.end(); ++it) { + for (const auto &[pid, _] : reps) { tasking::enqueue( LPC_CM_DISCONNECTED_SCATTER, &_tracker, - std::bind(&replica_stub::on_meta_server_disconnected_scatter, this, this, it->first), - it->first.thread_hash()); + std::bind(&replica_stub::on_meta_server_disconnected_scatter, this, this, pid), + pid.thread_hash()); } } @@ -2115,6 +2117,14 @@ replica *replica_stub::new_replica(gpid gpid, return rep; } +replica *replica_stub::new_replica(gpid gpid, + const app_info &app, + bool restore_if_necessary, + bool is_duplication_follower) +{ + return new_replica(gpid, app, restore_if_necessary, is_duplication_follower, ""); +} + /*static*/ std::string replica_stub::get_replica_dir_name(const std::string &dir) { static const char splitters[] = {'\\', '/', 0}; @@ -2124,15 +2134,28 @@ replica *replica_stub::new_replica(gpid gpid, /* static */ bool replica_stub::parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type) { - int32_t app_id = 0, partition_id = 0; - char app_type_buf[128] = {0}; - if (3 != sscanf(dir_name.c_str(), "%d.%d.%s", &app_id, &partition_id, app_type_buf)) { + std::vector ids(2, 0); + size_t begin = 0; + for (auto &id : ids) { + size_t end = dir_name.find('.', begin); + if (end == std::string::npos) { + return false; + } + + if (!buf2int32(std::string_view(dir_name.data() + begin, end - begin), id)) { + return false; + } + + begin = end + 1; + } + + if (begin >= dir_name.size()) { return false; } - pid.set_app_id(app_id); - pid.set_partition_index(partition_id); - app_type = app_type_buf; + pid.set_app_id(ids[0]); + pid.set_partition_index(ids[1]); + app_type.assign(dir_name, begin); return true; } diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index ded7b90ab8..c04f50f864 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -27,9 +27,9 @@ #pragma once #include -#include #include #include +#include #include #include #include @@ -71,11 +71,11 @@ #include "utils/metrics.h" #include "utils/zlocks.h" -namespace dsn { -namespace utils { +namespace dsn::utils { + class ex_lock; -} // namespace utils -} // namespace dsn + +} // namespace dsn::utils DSN_DECLARE_uint32(max_concurrent_manual_emergency_checkpointing_count); @@ -365,7 +365,12 @@ class replica_stub : public serverlet, public ref_counter const app_info &app, bool restore_if_necessary, bool is_duplication_follower, - const std::string &parent_dir = ""); + const std::string &parent_dir); + + replica *new_replica(gpid gpid, + const app_info &app, + bool restore_if_necessary, + bool is_duplication_follower); using disk_dirs = std::vector>>; @@ -479,7 +484,7 @@ class replica_stub : public serverlet, public ref_counter friend class replica_follower; friend class replica_follower_test; friend class replica_http_service_test; - friend class LoadReplicasTest; + friend class mock_load_replica; FRIEND_TEST(open_replica_test, open_replica_add_decree_and_ballot_check); FRIEND_TEST(replica_test, test_auto_trash_of_corruption); FRIEND_TEST(replica_test, test_clear_on_failure); @@ -600,5 +605,6 @@ class replica_stub : public serverlet, public ref_counter dsn::task_tracker _tracker; }; + } // namespace replication } // namespace dsn diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index f850b405ef..5234ae6df9 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. +#include +#include #include -#include -#include #include #include #include @@ -47,21 +47,14 @@ DSN_DECLARE_uint64(max_replicas_on_load_for_each_disk); -namespace dsn { -namespace replication { +namespace dsn::replication { -struct load_replicas_case -{ - std::map dirs_by_tag; - std::map> replicas_by_tag; -}; - -class LoadReplicasTest : public replica_stub, public testing::TestWithParam +class mock_load_replica : public replica_stub { public: - LoadReplicasTest() = default; + mock_load_replica() = default; - ~LoadReplicasTest() override = default; + ~mock_load_replica() override = default; void initialize(const std::map &dirs_by_tag, const std::map> &replicas_by_tag) @@ -127,8 +120,6 @@ class LoadReplicasTest : public replica_stub, public testing::TestWithParam dirs_by_tag; + std::map> replicas_by_tag; +}; -TEST_P(LoadReplicasTest, LoadOrder) +class LoadReplicasTest : public testing::TestWithParam { - const auto &load_case = GetParam(); - initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(true, 256); -} +public: + LoadReplicasTest() + { + const auto &load_case = GetParam(); + _stub.initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); + } -TEST_P(LoadReplicasTest, LoadThrottling) -{ - const auto &load_case = GetParam(); - initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); - test_load_replicas(false, 5); -} + ~LoadReplicasTest() override = default; + + void TearDown() override { _stub.remove_disk_dirs(); } + + void test_load_replicas(bool test_load_order, uint64_t max_replicas_on_load_for_each_disk) + { + _stub.test_load_replicas(test_load_order, max_replicas_on_load_for_each_disk); + } + +private: + mock_load_replica _stub; + + DISALLOW_COPY_AND_ASSIGN(LoadReplicasTest); +}; + +TEST_P(LoadReplicasTest, LoadReplicas) { test_load_replicas(false, 256); } + +TEST_P(LoadReplicasTest, LoadOrder) { test_load_replicas(true, 256); } + +TEST_P(LoadReplicasTest, LoadThrottling) { test_load_replicas(false, 5); } load_replicas_case generate_load_replicas_case(const std::vector &disk_replicas) { @@ -302,5 +310,4 @@ INSTANTIATE_TEST_SUITE_P(ReplicaStubTest, LoadReplicasTest, testing::ValuesIn(generate_load_replicas_cases())); -} // namespace replication -} // namespace dsn +} // namespace dsn::replication diff --git a/src/utils/time_utils.h b/src/utils/time_utils.h index 8730b5d7f4..872ea4fb70 100644 --- a/src/utils/time_utils.h +++ b/src/utils/time_utils.h @@ -147,7 +147,7 @@ class chronograph inline void reset_start_time() { _start_time_ns = dsn_now_ns(); } - inline uint64_t duration_ns() const + [[nodiscard]] inline uint64_t duration_ns() const { auto now = dsn_now_ns(); CHECK_GE(now, _start_time_ns); @@ -155,7 +155,7 @@ class chronograph return now - _start_time_ns; } - inline uint64_t duration_ms() const { return duration_ns() / 1'000'000; } + [[nodiscard]] inline uint64_t duration_ms() const { return duration_ns() / 1'000'000; } private: uint64_t _start_time_ns; From ff70d9fc3a5e17ba06099f39d5e5243c514b7b60 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 30 Aug 2024 18:06:13 +0800 Subject: [PATCH 19/43] fix clang tidy --- src/replica/test/load_replicas_test.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 5234ae6df9..7c75aa756b 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -44,6 +44,7 @@ #include "utils/autoref_ptr.h" #include "utils/filesystem.h" #include "utils/flags.h" +#include "utils/ports.h" DSN_DECLARE_uint64(max_replicas_on_load_for_each_disk); @@ -199,6 +200,7 @@ class mock_load_replica : public replica_stub replicas _expected_loaded_replicas; DISALLOW_COPY_AND_ASSIGN(mock_load_replica); + DISALLOW_MOVE_AND_ASSIGN(mock_load_replica); }; struct load_replicas_case @@ -229,6 +231,7 @@ class LoadReplicasTest : public testing::TestWithParam mock_load_replica _stub; DISALLOW_COPY_AND_ASSIGN(LoadReplicasTest); + DISALLOW_MOVE_AND_ASSIGN(LoadReplicasTest); }; TEST_P(LoadReplicasTest, LoadReplicas) { test_load_replicas(false, 256); } From f7ed2c913697b51453a51b36cc1a5a0486e262b8 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Mon, 2 Sep 2024 11:04:06 +0800 Subject: [PATCH 20/43] fix centos 7 compilation and IWYU --- src/replica/replica_stub.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 3b1e2ff319..80908b8985 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -2155,7 +2155,10 @@ replica_stub::parse_replica_dir_name(const std::string &dir_name, gpid &pid, std pid.set_app_id(ids[0]); pid.set_partition_index(ids[1]); - app_type.assign(dir_name, begin); + + // TODO(wangdan): the 3rd parameter `count` does not support default argument for CentOS 7 + // (gcc 7.3.1). After CentOS 7 is deprecated, consider dropping std::string::npos. + app_type.assign(dir_name, begin, std::string::npos); return true; } From 8f10ee93fc39d7780e8d354aeec838a4aff770b4 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Mon, 2 Sep 2024 11:04:59 +0800 Subject: [PATCH 21/43] fix IWYU --- src/replica/replica_stub.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 80908b8985..46b2dc8ac5 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -83,6 +83,7 @@ #include "utils/ports.h" #include "utils/process_utils.h" #include "utils/rand.h" +#include "utils/string_conv.h" #include "utils/strings.h" #include "utils/synchronize.h" #include "utils/threadpool_spec.h" From 7c9044182f585168bcb91da9c87f78d845a3465f Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Mon, 2 Sep 2024 16:04:43 +0800 Subject: [PATCH 22/43] fix clang tidy --- build_tools/clang_tidy.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build_tools/clang_tidy.py b/build_tools/clang_tidy.py index 07271b26a8..8a35e1a5dd 100755 --- a/build_tools/clang_tidy.py +++ b/build_tools/clang_tidy.py @@ -96,7 +96,7 @@ def tidy_on_path(path): "-readability-function-cognitive-complexity," "-readability-identifier-length," "-readability-magic-numbers," - "-readability-named-parameter", + "-readability-named-parameter," "-readability-suspicious-call-argument", "-extra-arg=-language=c++", "-extra-arg=-std=c++17", From d63ace34980765986d97960ea00019e1acd537b4 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Wed, 4 Sep 2024 17:49:21 +0800 Subject: [PATCH 23/43] rename parameter --- src/replica/replica_stub.cpp | 12 ++++++------ src/replica/replica_stub.h | 8 ++++---- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 46b2dc8ac5..9aaa189157 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -2213,7 +2213,7 @@ bool replica_stub::validate_replica_dir(const std::string &dir, return true; } -replica_ptr replica_stub::load_replica(dir_node *dn, const char *dir) +replica_ptr replica_stub::load_replica(dir_node *dn, const char *replica_dir) { FAIL_POINT_INJECT_F("mock_replica_load", [&](std::string_view) -> replica * { return nullptr; }); @@ -2221,13 +2221,13 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const char *dir) app_info ai; gpid pid; std::string hint_message; - if (!validate_replica_dir(dir, ai, pid, hint_message)) { - LOG_ERROR("invalid replica dir '{}', hint: {}", dir, hint_message); + if (!validate_replica_dir(replica_dir, ai, pid, hint_message)) { + LOG_ERROR("invalid replica dir '{}', hint: {}", replica_dir, hint_message); return nullptr; } // The replica's directory must exist when creating a replica. - CHECK_EQ(dir, dn->replica_dir(ai.app_type, pid)); + CHECK_EQ(replica_dir, dn->replica_dir(ai.app_type, pid)); auto *rep = new replica(this, pid, ai, dn, false); const auto err = rep->initialize_on_load(); if (err != ERR_OK) { @@ -2236,8 +2236,8 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const char *dir) rep = nullptr; // clear work on failure - if (dsn::utils::filesystem::directory_exists(dir)) { - move_to_err_path(dir, "load replica"); + if (dsn::utils::filesystem::directory_exists(replica_dir)) { + move_to_err_path(replica_dir, "load replica"); METRIC_VAR_INCREMENT(moved_error_replicas); _fs_manager.remove_replica(pid); } diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index c04f50f864..18b5f88370 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -385,10 +385,10 @@ class replica_stub : public serverlet, public ref_counter static bool parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type); - // Load an existing replica which is located in `dn` with `dir`. Usually each different - // `dn` represents a unique disk. `dir` is the absolute path of the directory for a - // replica. - virtual replica_ptr load_replica(dir_node *dn, const char *dir); + // Load an existing replica which is located in `dn` with `replica_dir`. Usually each + // different `dn` represents a unique disk. `dir` is the absolute path of the directory + // for a replica. + virtual replica_ptr load_replica(dir_node *dn, const char *replica_dir); // The same as the above `load_replica` function, except that this function is to load // each replica to `reps` with protection from `reps_lock`. From be38fb44bbee1377636e4e0cbe89f044cb814c33 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 20 Sep 2024 11:47:39 +0800 Subject: [PATCH 24/43] add GetReplicaDirNameTest --- src/replica/replica_stub.h | 1 + src/replica/test/replica_dir_test.cpp | 97 +++++++++++++++++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 src/replica/test/replica_dir_test.cpp diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index 18b5f88370..00de20857c 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -485,6 +485,7 @@ class replica_stub : public serverlet, public ref_counter friend class replica_follower_test; friend class replica_http_service_test; friend class mock_load_replica; + friend class GetReplicaDirNameTest; FRIEND_TEST(open_replica_test, open_replica_add_decree_and_ballot_check); FRIEND_TEST(replica_test, test_auto_trash_of_corruption); FRIEND_TEST(replica_test, test_clear_on_failure); diff --git a/src/replica/test/replica_dir_test.cpp b/src/replica/test/replica_dir_test.cpp new file mode 100644 index 0000000000..39d5a78165 --- /dev/null +++ b/src/replica/test/replica_dir_test.cpp @@ -0,0 +1,97 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/fs_manager.h" +#include "common/gpid.h" +#include "common/replication.codes.h" +#include "dsn.layer2_types.h" +#include "gtest/gtest.h" +#include "replica/replica.h" +#include "replica/replica_stub.h" +#include "replica/replication_app_base.h" +#include "replica/test/mock_utils.h" +#include "runtime/task/task.h" +#include "runtime/task/task_code.h" +#include "runtime/task/task_spec.h" +#include "test_util/test_util.h" +#include "utils/autoref_ptr.h" +#include "utils/filesystem.h" +#include "utils/flags.h" +#include "utils/ports.h" + +namespace dsn::replication { + +struct get_replica_dir_name_case +{ + std::string path; + std::string expected_replica_dir_name; +}; + +class GetReplicaDirNameTest : public testing::TestWithParam +{ +public: + GetReplicaDirNameTest() = default; + + ~GetReplicaDirNameTest() override = default; + + void test_get_replica_dir_name() + { + const auto &test_case = GetParam(); + const auto &actual_replica_dir_name = replica_stub::get_replica_dir_name(test_case.path); + EXPECT_EQ(test_case.expected_replica_dir_name, actual_replica_dir_name); + } +}; + +TEST_P(GetReplicaDirNameTest, GetReplicaDirName) { test_get_replica_dir_name(); } + +const std::vector get_replica_dir_name_tests{ + // Linux absolute path and non-empty dir name. + {"/data/pegasus/1.2.pegasus", "1.2.pegasus"}, + // Linux absolute path and empty dir name. + {"/data/pegasus/1.2.pegasus/", ""}, + // Windows absolute path and non-empty dir name. + {"D:\\data\\pegasus\\1.2.pegasus", "1.2.pegasus"}, + // Windows absolute path and empty dir name. + {"D:\\data\\pegasus\\1.2.pegasus\\", ""}, + // Linux relative path and non-empty dir name. + {"./1.2.pegasus", "1.2.pegasus"}, + // Linux relative path and empty dir name. + {"./1.2.pegasus/", ""}, + // Windows relative path and non-empty dir name. + {".\\1.2.pegasus", "1.2.pegasus"}, + // Windows relative path and empty dir name. + {".\\1.2.pegasus\\", ""}, +}; + +INSTANTIATE_TEST_SUITE_P(ReplicaDirTest, + GetReplicaDirNameTest, + testing::ValuesIn(get_replica_dir_name_tests)); + +} // namespace dsn::replication From 1400fc93e96ae55c431afd6338d513cd10b0856e Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 20 Sep 2024 13:37:13 +0800 Subject: [PATCH 25/43] add ParseReplicaDirNameTest, fix clang-tidy and IWYU --- src/replica/replica_stub.h | 1 + src/replica/test/replica_dir_test.cpp | 79 +++++++++++++++------------ 2 files changed, 46 insertions(+), 34 deletions(-) diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index 00de20857c..fdf5d81977 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -486,6 +486,7 @@ class replica_stub : public serverlet, public ref_counter friend class replica_http_service_test; friend class mock_load_replica; friend class GetReplicaDirNameTest; + friend class ParseReplicaDirNameTest; FRIEND_TEST(open_replica_test, open_replica_add_decree_and_ballot_check); FRIEND_TEST(replica_test, test_auto_trash_of_corruption); FRIEND_TEST(replica_test, test_clear_on_failure); diff --git a/src/replica/test/replica_dir_test.cpp b/src/replica/test/replica_dir_test.cpp index 39d5a78165..2d5349af8a 100644 --- a/src/replica/test/replica_dir_test.cpp +++ b/src/replica/test/replica_dir_test.cpp @@ -15,36 +15,11 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include #include -#include "common/fs_manager.h" -#include "common/gpid.h" -#include "common/replication.codes.h" -#include "dsn.layer2_types.h" #include "gtest/gtest.h" -#include "replica/replica.h" #include "replica/replica_stub.h" -#include "replica/replication_app_base.h" -#include "replica/test/mock_utils.h" -#include "runtime/task/task.h" -#include "runtime/task/task_code.h" -#include "runtime/task/task_spec.h" -#include "test_util/test_util.h" -#include "utils/autoref_ptr.h" -#include "utils/filesystem.h" -#include "utils/flags.h" -#include "utils/ports.h" namespace dsn::replication { @@ -57,11 +32,7 @@ struct get_replica_dir_name_case class GetReplicaDirNameTest : public testing::TestWithParam { public: - GetReplicaDirNameTest() = default; - - ~GetReplicaDirNameTest() override = default; - - void test_get_replica_dir_name() + static void test_get_replica_dir_name() { const auto &test_case = GetParam(); const auto &actual_replica_dir_name = replica_stub::get_replica_dir_name(test_case.path); @@ -77,21 +48,61 @@ const std::vector get_replica_dir_name_tests{ // Linux absolute path and empty dir name. {"/data/pegasus/1.2.pegasus/", ""}, // Windows absolute path and non-empty dir name. - {"D:\\data\\pegasus\\1.2.pegasus", "1.2.pegasus"}, + {R"(D:\data\pegasus\1.2.pegasus)", "1.2.pegasus"}, // Windows absolute path and empty dir name. - {"D:\\data\\pegasus\\1.2.pegasus\\", ""}, + {R"(D:\data\pegasus\1.2.pegasus\)", ""}, // Linux relative path and non-empty dir name. {"./1.2.pegasus", "1.2.pegasus"}, // Linux relative path and empty dir name. {"./1.2.pegasus/", ""}, // Windows relative path and non-empty dir name. - {".\\1.2.pegasus", "1.2.pegasus"}, + {R"(.\1.2.pegasus)", "1.2.pegasus"}, // Windows relative path and empty dir name. - {".\\1.2.pegasus\\", ""}, + {R"(.\1.2.pegasus\)", ""}, }; INSTANTIATE_TEST_SUITE_P(ReplicaDirTest, GetReplicaDirNameTest, testing::ValuesIn(get_replica_dir_name_tests)); +struct parse_replica_dir_name_case +{ + std::string replica_dir_name; + bool ok; + gpid expected_pid; + std::string expected_app_type; +}; + +class ParseReplicaDirNameTest : public testing::TestWithParam +{ +public: + static void test_parse_replica_dir_name() + { + const auto &test_case = GetParam(); + + gpid actual_pid; + std::string actual_app_type; + ASSERT_EQ(test_case.ok, + replica_stub::parse_replica_dir_name( + test_case.replica_dir_name, actual_pid, actual_app_type)); + if (!test_case.ok) { + return; + } + + EXPECT_EQ(test_case.expected_pid, actual_pid); + EXPECT_EQ(test_case.expected_app_type, actual_app_type); + } +}; + +TEST_P(ParseReplicaDirNameTest, ParseReplicaDirName) { test_parse_replica_dir_name(); } + +const std::vector parse_replica_dir_name_tests{ + {"", false, {}, ""}, + {"1.2.pegasus", true, {1, 2}, "pegasus"}, +}; + +INSTANTIATE_TEST_SUITE_P(ReplicaDirTest, + ParseReplicaDirNameTest, + testing::ValuesIn(parse_replica_dir_name_tests)); + } // namespace dsn::replication From f06eef729db1e7d5718c46291f8c6375e1b4c8eb Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 20 Sep 2024 15:30:46 +0800 Subject: [PATCH 26/43] fix ParseReplicaDirNameTest and fix IWYU --- src/replica/replica_stub.cpp | 8 ++++---- src/replica/test/replica_dir_test.cpp | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 9aaa189157..a6b46196ab 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -2135,7 +2135,7 @@ replica *replica_stub::new_replica(gpid gpid, /* static */ bool replica_stub::parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type) { - std::vector ids(2, 0); + std::vector ids(2, 0); size_t begin = 0; for (auto &id : ids) { size_t end = dir_name.find('.', begin); @@ -2143,7 +2143,7 @@ replica_stub::parse_replica_dir_name(const std::string &dir_name, gpid &pid, std return false; } - if (!buf2int32(std::string_view(dir_name.data() + begin, end - begin), id)) { + if (!buf2uint32(std::string_view(dir_name.data() + begin, end - begin), id)) { return false; } @@ -2154,8 +2154,8 @@ replica_stub::parse_replica_dir_name(const std::string &dir_name, gpid &pid, std return false; } - pid.set_app_id(ids[0]); - pid.set_partition_index(ids[1]); + pid.set_app_id(static_cast(ids[0])); + pid.set_partition_index(static_cast(ids[1])); // TODO(wangdan): the 3rd parameter `count` does not support default argument for CentOS 7 // (gcc 7.3.1). After CentOS 7 is deprecated, consider dropping std::string::npos. diff --git a/src/replica/test/replica_dir_test.cpp b/src/replica/test/replica_dir_test.cpp index 2d5349af8a..3bc64d74f8 100644 --- a/src/replica/test/replica_dir_test.cpp +++ b/src/replica/test/replica_dir_test.cpp @@ -18,6 +18,7 @@ #include #include +#include "common/gpid.h" #include "gtest/gtest.h" #include "replica/replica_stub.h" @@ -97,8 +98,26 @@ class ParseReplicaDirNameTest : public testing::TestWithParam parse_replica_dir_name_tests{ + // Empty dir name. {"", false, {}, ""}, + // Single-digit IDs. {"1.2.pegasus", true, {1, 2}, "pegasus"}, + // Multi-digit IDs. + {"1234.56789.pegasus", true, {1234, 56789}, "pegasus"}, + // Custom app type other than "pegasus". + {"1.2.another", true, {1, 2}, "another"}, + // Custom app type with dot. + {"1.2.another.pegasus", true, {1, 2}, "another.pegasus"}, + // Custom app type with other specific symbol. + {"1.2.another_pegasus", true, {1, 2}, "another_pegasus"}, + // Missing one ID. + {"1.pegasus", false, {}, ""}, + // Missing both IDs. + {"pegasus", false, {}, ""}, + // ID with letter. + {"1.2a.pegasus", false, {}, ""}, + // ID with minus. + {"1.-2.pegasus", false, {}, ""}, }; INSTANTIATE_TEST_SUITE_P(ReplicaDirTest, From 1116e0f4d753f46cc2cbc99e0a32ea8a24fab0ce Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 20 Sep 2024 16:44:23 +0800 Subject: [PATCH 27/43] fix clang tidy --- src/replica/test/load_replicas_test.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 7c75aa756b..4f927500f7 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -37,9 +37,9 @@ #include "replica/replica_stub.h" #include "replica/replication_app_base.h" #include "replica/test/mock_utils.h" -#include "runtime/task/task.h" -#include "runtime/task/task_code.h" -#include "runtime/task/task_spec.h" +#include "task/task.h" +#include "task/task_code.h" +#include "task/task_spec.h" #include "test_util/test_util.h" #include "utils/autoref_ptr.h" #include "utils/filesystem.h" From 1885144bbd97fc938e7600b7b25267025b5eaa39 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Thu, 26 Sep 2024 21:13:17 +0800 Subject: [PATCH 28/43] fix load replicas --- src/replica/replica_stub.cpp | 35 +++++++++++++++-------------------- src/replica/replica_stub.h | 5 +---- 2 files changed, 16 insertions(+), 24 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index a6b46196ab..e889908640 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -493,8 +493,8 @@ void replica_stub::load_replica(dir_node *dn, const auto *const worker = task::get_current_worker2(); if (worker != nullptr) { CHECK(!(worker->pool()->spec().partitioned), - "The thread pool for loading replicas must not be partitioned since load balancing " - "is required among multiple threads"); + "The thread pool LPC_REPLICATION_INIT_LOAD for loading replicas must not be " + "partitioned since load balancing is required among multiple threads"); } auto rep = load_replica(dn, dir.c_str()); @@ -513,12 +513,13 @@ void replica_stub::load_replica(dir_node *dn, rep->last_prepared_decree()); utils::auto_lock l(reps_lock); - CHECK(reps.find(rep->get_gpid()) == reps.end(), + const auto rep_iter = reps.find(rep->get_gpid()); + CHECK(rep_iter == reps.end(), "conflict replica dir: {} <--> {}", rep->dir(), - reps[rep->get_gpid()]->dir()); + rep_iter->second->dir()); - reps[rep->get_gpid()] = rep; + reps.emplace(rep->get_gpid(), rep); } void replica_stub::load_replicas(replicas &reps) @@ -556,13 +557,10 @@ void replica_stub::load_replicas(replicas &reps) const auto &dn = disks[disk_index].first; auto &load_disk_queue = load_disk_queues[disk_index]; - if (!load_disk_queue.empty() && - load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { + if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { // Loading replicas should be throttled in case that disk IO is saturated. - if (load_disk_queue.front().second->wait( + if (!load_disk_queue.front().second->wait( static_cast(FLAGS_load_replica_max_wait_time_ms))) { - load_disk_queue.pop(); - } else { // There might be too many replicas that are being loaded which lead to // slow disk IO. LOG_WARNING("after {} ms, loading dir({}) is still not finished, there are " @@ -580,10 +578,13 @@ void replica_stub::load_replicas(replicas &reps) } // Continue to load a replica since we are within the limit now. - if (dsn_unlikely(load_disk_queue.size() >= - FLAGS_max_replicas_on_load_for_each_disk)) { - continue; - } + load_disk_queue.pop(); + } + + const auto &dir = dirs[dir_index++]; + if (dsn::replication::is_data_dir_invalid(dir)) { + LOG_WARNING("ignore dir {}", dir); + continue; } LOG_DEBUG("ready to load dir(index={}, path={}) for disk(index={}, tag={}, path={})", @@ -593,12 +594,6 @@ void replica_stub::load_replicas(replicas &reps) dn->tag, dn->full_dir); - const auto &dir = dirs[dir_index++]; - if (dsn::replication::is_data_dir_invalid(dir)) { - LOG_WARNING("ignore dir {}", dir); - continue; - } - load_disk_queue.emplace( dir, tasking::create_task( diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index fdf5d81977..cc15987aca 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -395,10 +395,7 @@ class replica_stub : public serverlet, public ref_counter void load_replica(dir_node *dn, const std::string &dir, utils::ex_lock &reps_lock, replicas &reps); - // Load all replicas synchronously from all disks to `reps`. This function would ensure - // that data on each disk is loaded more evenly, rather than that a disk would begin to - // be loaded only after another has been finished, in case that there are too many replicas - // on a disk and other disks cannot start loading until this disk is finished. + // Load all replicas simultaneously from all disks to `reps`. void load_replicas(replicas &reps); // Clean up the memory state and on disk data if creating replica failed. From 1d8a460b347a02dc2a2b301357e60c55884550fc Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 27 Sep 2024 12:00:17 +0800 Subject: [PATCH 29/43] fix upload-artifact --- .github/actions/upload_artifact/action.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/actions/upload_artifact/action.yaml b/.github/actions/upload_artifact/action.yaml index a58606f9e8..e62abe4a3d 100644 --- a/.github/actions/upload_artifact/action.yaml +++ b/.github/actions/upload_artifact/action.yaml @@ -30,7 +30,7 @@ runs: tar -zcvhf ${ARTIFACT_NAME}_builder.tar build/latest/output build/latest/bin build/latest/src/server/test/config.ini hadoop-bin zookeeper-bin shell: bash - name: Upload tarball - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: ${{ env.ARTIFACT_NAME }}_artifact_${{ github.sha }} path: ${{ env.ARTIFACT_NAME }}_builder.tar From ae9445ab74092c8319f2425181c65cd68ad01061 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Mon, 16 Dec 2024 14:25:39 +0800 Subject: [PATCH 30/43] rollback upload-artifact --- .github/actions/upload_artifact/action.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/actions/upload_artifact/action.yaml b/.github/actions/upload_artifact/action.yaml index e62abe4a3d..a58606f9e8 100644 --- a/.github/actions/upload_artifact/action.yaml +++ b/.github/actions/upload_artifact/action.yaml @@ -30,7 +30,7 @@ runs: tar -zcvhf ${ARTIFACT_NAME}_builder.tar build/latest/output build/latest/bin build/latest/src/server/test/config.ini hadoop-bin zookeeper-bin shell: bash - name: Upload tarball - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v3 with: name: ${{ env.ARTIFACT_NAME }}_artifact_${{ github.sha }} path: ${{ env.ARTIFACT_NAME }}_builder.tar From ae081265d4de0547b7edc41f09d07d689e858b7b Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Mon, 16 Dec 2024 20:47:39 +0800 Subject: [PATCH 31/43] fix load replicas and tests --- src/replica/replica_stub.cpp | 43 ++++++++++++++++--------- src/replica/test/load_replicas_test.cpp | 6 ++-- 2 files changed, 31 insertions(+), 18 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index e889908640..26ea875fde 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -527,7 +527,15 @@ void replica_stub::load_replicas(replicas &reps) const auto &disks = get_all_disk_dirs(); std::vector dir_indexes(disks.size(), 0); - std::vector>> load_disk_queues(disks.size()); + + struct replica_dir_loader + { + size_t dir_index; + std::string dir_path; + task_ptr load_task; + }; + std::vector> load_disk_queues(disks.size()); + utils::ex_lock reps_lock; while (true) { @@ -559,15 +567,17 @@ void replica_stub::load_replicas(replicas &reps) auto &load_disk_queue = load_disk_queues[disk_index]; if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { // Loading replicas should be throttled in case that disk IO is saturated. - if (!load_disk_queue.front().second->wait( + if (!load_disk_queue.front().load_task->wait( static_cast(FLAGS_load_replica_max_wait_time_ms))) { // There might be too many replicas that are being loaded which lead to // slow disk IO. - LOG_WARNING("after {} ms, loading dir({}) is still not finished, there are " - "{} replicas being loaded for disk(index={}, tag={}, path={}), " - "skip dir(index={}, path={}), turn to next disk", + LOG_WARNING("after {} ms, loading dir(index={}, path={}) is still not " + "finished, there are {} replicas being loaded for disk(index" + "={}, tag={}, path={}), skip dir(index={}, path={}), turn to " + "next disk", FLAGS_load_replica_max_wait_time_ms, - load_disk_queue.front().first, + load_disk_queue.front().dir_index, + load_disk_queue.front().dir_path, load_disk_queue.size(), disk_index, dn->tag, @@ -581,9 +591,9 @@ void replica_stub::load_replicas(replicas &reps) load_disk_queue.pop(); } - const auto &dir = dirs[dir_index++]; - if (dsn::replication::is_data_dir_invalid(dir)) { - LOG_WARNING("ignore dir {}", dir); + if (dsn::replication::is_data_dir_invalid(dirs[dir_index])) { + LOG_WARNING("ignore dir(index={}, path={})", dir_index, dirs[dir_index]); + ++dir_index; continue; } @@ -594,8 +604,9 @@ void replica_stub::load_replicas(replicas &reps) dn->tag, dn->full_dir); - load_disk_queue.emplace( - dir, + load_disk_queue.push(replica_dir_loader{ + dir_index, + dirs[dir_index], tasking::create_task( // Ensure that the thread pool is non-partitioned. LPC_REPLICATION_INIT_LOAD, @@ -605,11 +616,13 @@ void replica_stub::load_replicas(replicas &reps) &replica_stub::load_replica), this, dn, - dir, + dirs[dir_index], std::ref(reps_lock), - std::ref(reps)))); + std::ref(reps)))}); + + load_disk_queue.back().load_task->enqueue(); - load_disk_queue.back().second->enqueue(); + ++dir_index; } if (finished_disks >= disks.size()) { @@ -621,7 +634,7 @@ void replica_stub::load_replicas(replicas &reps) // All loading tasks have been in the queue. Just wait all tasks to be finished. for (auto &load_disk_queue : load_disk_queues) { while (!load_disk_queue.empty()) { - CHECK_TRUE(load_disk_queue.front().second->wait()); + CHECK_TRUE(load_disk_queue.front().load_task->wait()); load_disk_queue.pop(); } } diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 4f927500f7..1a5ee710c3 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -214,13 +214,13 @@ class LoadReplicasTest : public testing::TestWithParam public: LoadReplicasTest() { + _stub.remove_disk_dirs(); + const auto &load_case = GetParam(); _stub.initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); } - ~LoadReplicasTest() override = default; - - void TearDown() override { _stub.remove_disk_dirs(); } + ~LoadReplicasTest() override { _stub.remove_disk_dirs(); } void test_load_replicas(bool test_load_order, uint64_t max_replicas_on_load_for_each_disk) { From 871b0041e661e446ab92f1e3590dcaf4347d905d Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Tue, 17 Dec 2024 19:02:16 +0800 Subject: [PATCH 32/43] refactor --- src/replica/replica_stub.cpp | 27 ++++++++++++++++----------- src/replica/replica_stub.h | 11 +++++++++-- 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 26ea875fde..5bad341142 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -464,9 +464,9 @@ void replica_stub::initialize(bool clear /* = false*/) _access_controller = std::make_unique(); } -replica_stub::disk_dirs replica_stub::get_all_disk_dirs() const +std::vector replica_stub::get_all_disk_dirs() const { - disk_dirs disks; + std::vector disks; for (const auto &dn : _fs_manager.get_dir_nodes()) { if (dsn_unlikely(dn->status == disk_status::IO_ERROR)) { // Skip disks with IO errors. @@ -477,12 +477,14 @@ replica_stub::disk_dirs replica_stub::get_all_disk_dirs() const CHECK(dsn::utils::filesystem::get_subdirectories(dn->full_dir, sub_dirs, false), "failed to get sub_directories in {}", dn->full_dir); - disks.emplace_back(dn.get(), std::move(sub_dirs)); + disks.push_back(disk_replicas_info{dn.get(), std::move(sub_dirs)}); } return disks; } +// TaskCode: LPC_REPLICATION_INIT_LOAD +// ThreadPool: THREAD_POOL_LOCAL_APP void replica_stub::load_replica(dir_node *dn, const std::string &dir, utils::ex_lock &reps_lock, @@ -493,8 +495,9 @@ void replica_stub::load_replica(dir_node *dn, const auto *const worker = task::get_current_worker2(); if (worker != nullptr) { CHECK(!(worker->pool()->spec().partitioned), - "The thread pool LPC_REPLICATION_INIT_LOAD for loading replicas must not be " - "partitioned since load balancing is required among multiple threads"); + "The thread pool THREAD_POOL_LOCAL_APP(task code: LPC_REPLICATION_INIT_LOAD) " + "for loading replicas must not be partitioned since load balancing is required " + "among multiple threads"); } auto rep = load_replica(dn, dir.c_str()); @@ -544,16 +547,17 @@ void replica_stub::load_replicas(replicas &reps) // For each round, start loading one replica for each disk in case there are too many // replicas in a disk, except that all of the replicas of this disk are being loaded. for (size_t disk_index = 0; disk_index < disks.size(); ++disk_index) { - // Structured bindings can be captured by closures in g++, while not supported - // well by clang. Thus we do not use following statement to bind both variables - // until clang has been upgraded to version 16 which could support that well: + // TODO(wangdan): Structured bindings can be captured by closures in g++, while + // not supported well by clang. Thus we do not use following statement to bind + // both variables until clang has been upgraded to version 16 which could support + // that well: // // const auto &[dn, dirs] = disks[disk_index]; // // For the docs of clang 16 please see: // // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support. - const auto &dirs = disks[disk_index].second; + const auto &dirs = disks[disk_index].replica_dirs; auto &dir_index = dir_indexes[disk_index]; if (dir_index >= dirs.size()) { @@ -563,7 +567,7 @@ void replica_stub::load_replicas(replicas &reps) continue; } - const auto &dn = disks[disk_index].first; + const auto &dn = disks[disk_index].disk_node; auto &load_disk_queue = load_disk_queues[disk_index]; if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { // Loading replicas should be throttled in case that disk IO is saturated. @@ -2235,7 +2239,8 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const char *replica_dir) } // The replica's directory must exist when creating a replica. - CHECK_EQ(replica_dir, dn->replica_dir(ai.app_type, pid)); + CHECK_EQ(dn->replica_dir(ai.app_type, pid), replica_dir); + auto *rep = new replica(this, pid, ai, dn, false); const auto err = rep->initialize_on_load(); if (err != ERR_OK) { diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index cc15987aca..2a4c796075 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -372,10 +372,17 @@ class replica_stub : public serverlet, public ref_counter bool restore_if_necessary, bool is_duplication_follower); - using disk_dirs = std::vector>>; + struct disk_replicas_info + { + // `dir_node` for each disk. + dir_node *disk_node; + + // All replica dirs on each disk. + std::vector replica_dirs; + }; // Get the absolute dirs of all replicas for all disks. - disk_dirs get_all_disk_dirs() const; + std::vector get_all_disk_dirs() const; // Get the dir name for a replica from a potentially longer path (both absolute and // relative paths are possible). From 6329ecb9572c6c577e2883fb6c61349c9208efb0 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Wed, 18 Dec 2024 12:16:23 +0800 Subject: [PATCH 33/43] refactor --- src/replica/replica_stub.cpp | 26 ++++++++++++++----------- src/replica/replica_stub.h | 12 +++++++----- src/replica/test/load_replicas_test.cpp | 12 ++++++------ 3 files changed, 28 insertions(+), 22 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 5bad341142..6b21ba83fd 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -486,11 +486,11 @@ std::vector replica_stub::get_all_disk_dirs() // TaskCode: LPC_REPLICATION_INIT_LOAD // ThreadPool: THREAD_POOL_LOCAL_APP void replica_stub::load_replica(dir_node *dn, - const std::string &dir, + const std::string &replica_dir, utils::ex_lock &reps_lock, replicas &reps) { - LOG_INFO("loading replica: tag={}, dir={}", dn->tag, dir); + LOG_INFO("loading replica: tag={}, replica_dir={}", dn->tag, replica_dir); const auto *const worker = task::get_current_worker2(); if (worker != nullptr) { @@ -500,17 +500,17 @@ void replica_stub::load_replica(dir_node *dn, "among multiple threads"); } - auto rep = load_replica(dn, dir.c_str()); + auto rep = load_replica(dn, replica_dir); if (rep == nullptr) { return; } - LOG_INFO("{}@{}: load replica successfully, tag={}, dir={}, last_durable_decree={}, " + LOG_INFO("{}@{}: load replica successfully, tag={}, replica_dir={}, last_durable_decree={}, " "last_committed_decree={}, last_prepared_decree={}", rep->get_gpid(), dsn_primary_host_port(), dn->tag, - dir, + replica_dir, rep->last_durable_decree(), rep->last_committed_decree(), rep->last_prepared_decree()); @@ -1972,7 +1972,7 @@ void replica_stub::open_replica( _primary_host_port_cache, group_check ? "with" : "without", dir); - rep = load_replica(dn, dir.c_str()); + rep = load_replica(dn, dir); // if load data failed, re-open the `*.ori` folder which is the origin replica dir of disk // migration @@ -1997,7 +1997,7 @@ void replica_stub::open_replica( boost::replace_first( origin_dir, replica_disk_migrator::kReplicaDirOriginSuffix, ""); dsn::utils::filesystem::rename_path(origin_tmp_dir, origin_dir); - rep = load_replica(origin_dn, origin_dir.c_str()); + rep = load_replica(origin_dn, origin_dir); FAIL_POINT_INJECT_F("mock_replica_load", [&](std::string_view) -> void {}); } @@ -2225,7 +2225,7 @@ bool replica_stub::validate_replica_dir(const std::string &dir, return true; } -replica_ptr replica_stub::load_replica(dir_node *dn, const char *replica_dir) +replica_ptr replica_stub::load_replica(dir_node *dn, const std::string &replica_dir) { FAIL_POINT_INJECT_F("mock_replica_load", [&](std::string_view) -> replica * { return nullptr; }); @@ -2234,7 +2234,7 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const char *replica_dir) gpid pid; std::string hint_message; if (!validate_replica_dir(replica_dir, ai, pid, hint_message)) { - LOG_ERROR("invalid replica dir '{}', hint: {}", replica_dir, hint_message); + LOG_ERROR("invalid replica dir '{}', hint={}", replica_dir, hint_message); return nullptr; } @@ -2244,7 +2244,11 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const char *replica_dir) auto *rep = new replica(this, pid, ai, dn, false); const auto err = rep->initialize_on_load(); if (err != ERR_OK) { - LOG_ERROR("{}: load replica failed, err = {}", rep->name(), err); + LOG_ERROR("{}: load replica failed, tag={}, replica_dir={}, err={}", + rep->name(), + dn->tag, + replica_dir, + err); delete rep; rep = nullptr; @@ -2258,7 +2262,7 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const char *replica_dir) return nullptr; } - LOG_INFO("{}: load replica succeed", rep->name()); + LOG_INFO("{}: load replica succeed, tag={}, replica_dir={}", rep->name(), dn->tag, replica_dir); return rep; } diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index 2a4c796075..dc6ac4074e 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -393,14 +393,16 @@ class replica_stub : public serverlet, public ref_counter parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type); // Load an existing replica which is located in `dn` with `replica_dir`. Usually each - // different `dn` represents a unique disk. `dir` is the absolute path of the directory - // for a replica. - virtual replica_ptr load_replica(dir_node *dn, const char *replica_dir); + // different `dn` represents a unique disk. `replica_dir` is the absolute path of the + // directory for a replica. + virtual replica_ptr load_replica(dir_node *dn, const std::string &replica_dir); // The same as the above `load_replica` function, except that this function is to load // each replica to `reps` with protection from `reps_lock`. - void - load_replica(dir_node *dn, const std::string &dir, utils::ex_lock &reps_lock, replicas &reps); + void load_replica(dir_node *dn, + const std::string &replica_dir, + utils::ex_lock &reps_lock, + replicas &reps); // Load all replicas simultaneously from all disks to `reps`. void load_replicas(replicas &reps); diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 1a5ee710c3..a6ca7aecc3 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -122,11 +122,11 @@ class mock_load_replica : public replica_stub } private: - void load_replica_for_test(dir_node *dn, const char *dir, replica_ptr &rep) + void load_replica_for_test(dir_node *dn, const std::string &replica_dir, replica_ptr &rep) { - ASSERT_TRUE(utils::filesystem::directory_exists(dir)); + ASSERT_TRUE(utils::filesystem::directory_exists(replica_dir)); - const auto &dir_name = get_replica_dir_name(dir); + const auto &dir_name = get_replica_dir_name(replica_dir); gpid pid; std::string app_type; @@ -165,7 +165,7 @@ class mock_load_replica : public replica_stub } // Check the absolute dir of this replica. - ASSERT_EQ(dn->replica_dir("pegasus", pid), dir); + ASSERT_EQ(dn->replica_dir("pegasus", pid), replica_dir); app_info ai; ai.app_type = "pegasus"; @@ -180,10 +180,10 @@ class mock_load_replica : public replica_stub } // Mock the process of loading a replica. - replica_ptr load_replica(dir_node *dn, const char *dir) override + replica_ptr load_replica(dir_node *dn, const std::string &replica_dir) override { replica_ptr rep; - load_replica_for_test(dn, dir, rep); + load_replica_for_test(dn, replica_dir, rep); return rep; } From be70e2e0a6129eb991fda9b6e4913d000a759968 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Wed, 18 Dec 2024 14:54:24 +0800 Subject: [PATCH 34/43] rename replicas type --- src/replica/replica_stub.cpp | 32 ++++++++++++---------- src/replica/replica_stub.h | 36 +++++++++++++------------ src/replica/test/load_replicas_test.cpp | 4 +-- 3 files changed, 39 insertions(+), 33 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 6b21ba83fd..070f843f21 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -488,7 +488,7 @@ std::vector replica_stub::get_all_disk_dirs() void replica_stub::load_replica(dir_node *dn, const std::string &replica_dir, utils::ex_lock &reps_lock, - replicas &reps) + replica_map_by_gpid &reps) { LOG_INFO("loading replica: tag={}, replica_dir={}", dn->tag, replica_dir); @@ -518,14 +518,16 @@ void replica_stub::load_replica(dir_node *dn, utils::auto_lock l(reps_lock); const auto rep_iter = reps.find(rep->get_gpid()); CHECK(rep_iter == reps.end(), - "conflict replica dir: {} <--> {}", + "{}@{}: newly loaded dir {} conflicts with existing {} while loading replica", + rep->get_gpid(), + dsn_primary_host_port(), rep->dir(), rep_iter->second->dir()); reps.emplace(rep->get_gpid(), rep); } -void replica_stub::load_replicas(replicas &reps) +void replica_stub::load_replicas(replica_map_by_gpid &reps) { const auto &disks = get_all_disk_dirs(); @@ -615,8 +617,10 @@ void replica_stub::load_replicas(replicas &reps) // Ensure that the thread pool is non-partitioned. LPC_REPLICATION_INIT_LOAD, &_tracker, - std::bind(static_cast( + std::bind(static_cast( &replica_stub::load_replica), this, dn, @@ -729,7 +733,7 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f // Start to load replicas in available data directories. LOG_INFO("start to load replicas"); - replicas reps; + replica_map_by_gpid reps; utils::chronograph chrono; load_replicas(reps); @@ -894,7 +898,7 @@ dsn::error_code replica_stub::on_kill_replica(gpid id) { LOG_INFO("kill replica: gpid = {}", id); if (id.get_app_id() == -1 || id.get_partition_index() == -1) { - replicas rs; + replica_map_by_gpid rs; { zauto_read_lock l(_replicas_lock); rs = _replicas; @@ -1521,7 +1525,7 @@ void replica_stub::on_node_query_reply(error_code err, resp.partitions.size(), resp.gc_replicas.size()); - replicas reps; + replica_map_by_gpid reps; { zauto_read_lock rl(_replicas_lock); reps = _replicas; @@ -1671,7 +1675,7 @@ void replica_stub::on_meta_server_disconnected() _state = NS_Disconnected; - replicas reps; + replica_map_by_gpid reps; { zauto_read_lock rl(_replicas_lock); reps = _replicas; @@ -2064,7 +2068,7 @@ void replica_stub::open_replica( METRIC_VAR_DECREMENT(opening_replicas); CHECK(_replicas.find(id) == _replicas.end(), "replica {} is already in _replicas", id); - _replicas.insert(replicas::value_type(rep->get_gpid(), rep)); + _replicas.insert(replica_map_by_gpid::value_type(rep->get_gpid(), rep)); METRIC_VAR_INCREMENT(total_replicas); _closed_replicas.erase(id); @@ -2645,14 +2649,14 @@ replica_stub::exec_command_on_replica(const std::vector &args, return std::string("invalid arguments"); } - replicas rs; + replica_map_by_gpid rs; { zauto_read_lock l(_replicas_lock); rs = _replicas; } std::set required_ids; - replicas choosed_rs; + replica_map_by_gpid choosed_rs; if (!args.empty()) { for (int i = 0; i < args.size(); i++) { std::vector arg_strs; @@ -2915,7 +2919,7 @@ replica_ptr replica_stub::create_child_replica_if_not_found(gpid child_pid, CHECK_NOTNULL(dn, ""); auto *rep = new replica(this, child_pid, *app, dn, false); rep->_config.status = partition_status::PS_INACTIVE; - _replicas.insert(replicas::value_type(child_pid, rep)); + _replicas.insert(replica_map_by_gpid::value_type(child_pid, rep)); LOG_INFO("mock create_child_replica_if_not_found succeed"); return rep; }); @@ -2934,7 +2938,7 @@ replica_ptr replica_stub::create_child_replica_if_not_found(gpid child_pid, } else { replica *rep = new_replica(child_pid, *app, false, false, parent_dir); if (rep != nullptr) { - auto pr = _replicas.insert(replicas::value_type(child_pid, rep)); + auto pr = _replicas.insert(replica_map_by_gpid::value_type(child_pid, rep)); CHECK(pr.second, "child replica {} has been existed", rep->name()); METRIC_VAR_INCREMENT(total_replicas); _closed_replicas.erase(child_pid); diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index dc6ac4074e..c0bb7f43b6 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -27,7 +27,6 @@ #pragma once #include -#include #include #include #include @@ -122,11 +121,10 @@ class test_checker; } class cold_backup_context; class replica_split_manager; -typedef std::function - replica_state_subscriber; -typedef std::unordered_map replicas; + +using replica_state_subscriber = std::function; class replica_stub; @@ -397,15 +395,17 @@ class replica_stub : public serverlet, public ref_counter // directory for a replica. virtual replica_ptr load_replica(dir_node *dn, const std::string &replica_dir); + using replica_map_by_gpid = std::unordered_map; + // The same as the above `load_replica` function, except that this function is to load // each replica to `reps` with protection from `reps_lock`. void load_replica(dir_node *dn, const std::string &replica_dir, utils::ex_lock &reps_lock, - replicas &reps); + replica_map_by_gpid &reps); // Load all replicas simultaneously from all disks to `reps`. - void load_replicas(replicas &reps); + void load_replicas(replica_map_by_gpid &reps); // Clean up the memory state and on disk data if creating replica failed. void clear_on_failure(replica *rep); @@ -497,17 +497,19 @@ class replica_stub : public serverlet, public ref_counter FRIEND_TEST(replica_test, test_auto_trash_of_corruption); FRIEND_TEST(replica_test, test_clear_on_failure); - typedef std::unordered_map opening_replicas; - typedef std::unordered_map> - closing_replicas; // > - typedef std::map> - closed_replicas; // > + using opening_replica_map_by_gpid = std::unordered_map; + + // `task_ptr` is the task being closed. + using closing_replica_map_by_gpid = + std::unordered_map>; + + using closed_replica_map_by_gpid = std::map>; mutable zrwlock_nr _replicas_lock; - replicas _replicas; - opening_replicas _opening_replicas; - closing_replicas _closing_replicas; - closed_replicas _closed_replicas; + replica_map_by_gpid _replicas; + opening_replica_map_by_gpid _opening_replicas; + closing_replica_map_by_gpid _closing_replicas; + closed_replica_map_by_gpid _closed_replicas; ::dsn::host_port _primary_host_port; // The stringify of '_primary_host_port', used by logging usually. diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index a6ca7aecc3..2e6c08808a 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -103,7 +103,7 @@ class mock_load_replica : public replica_stub PRESERVE_FLAG(max_replicas_on_load_for_each_disk); FLAGS_max_replicas_on_load_for_each_disk = max_replicas_on_load_for_each_disk; - replicas actual_loaded_replicas; + replica_stub::replica_map_by_gpid actual_loaded_replicas; load_replicas(actual_loaded_replicas); ASSERT_EQ(_expected_loaded_replicas, actual_loaded_replicas); @@ -197,7 +197,7 @@ class mock_load_replica : public replica_stub std::vector _disk_loaded_replicas_for_order; mutable std::mutex _mtx; - replicas _expected_loaded_replicas; + replica_stub::replica_map_by_gpid _expected_loaded_replicas; DISALLOW_COPY_AND_ASSIGN(mock_load_replica); DISALLOW_MOVE_AND_ASSIGN(mock_load_replica); From 510dc78ac488f06b115b60a8fbc93d0b7dcf11eb Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Wed, 18 Dec 2024 20:42:03 +0800 Subject: [PATCH 35/43] fix clang-tidy and add command to update configurations for loading replicas dynamically --- src/replica/replica_stub.cpp | 155 ++++++++++++++++++++++------------- src/replica/replica_stub.h | 3 +- src/utils/command_manager.h | 41 +++++---- 3 files changed, 127 insertions(+), 72 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 070f843f21..a01105db84 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -97,8 +97,18 @@ #include "remote_cmd/remote_command.h" #include "utils/fail_point.h" -static const char *kMaxConcurrentBulkLoadDownloadingCountDesc = - "The maximum concurrent bulk load downloading replica count"; +namespace { + +const char *kMaxConcurrentBulkLoadDownloadingCountDesc = + "The maximum concurrent bulk load downloading replica count."; + +const char *kMaxReplicasOnLoadForEachDiskDesc = + "The max number of replicas that are allowed to be loaded simultaneously for each disk dir."; + +const char *kLoadReplicaMaxWaitTimeMsDesc = "The max waiting time for replica loading to complete."; + +} // anonymous namespace + DSN_DEFINE_int32(replication, max_concurrent_bulk_load_downloading_count, 5, @@ -269,18 +279,15 @@ DSN_DECLARE_string(server_key); DSN_DEFINE_uint64(replication, max_replicas_on_load_for_each_disk, 256, - "The max number of replicas that are allowed to be loaded simultaneously " - "for each disk dir."); + kMaxReplicasOnLoadForEachDiskDesc); DSN_TAG_VARIABLE(max_replicas_on_load_for_each_disk, FT_MUTABLE); DSN_DEFINE_validator(max_replicas_on_load_for_each_disk, - [](uint64_t max_replicas_on_load_for_each_disk) -> bool { - return max_replicas_on_load_for_each_disk > 0; - }); + [](uint64_t value) -> bool { return value > 0; }); -DSN_DEFINE_uint64(replication, - load_replica_max_wait_time_ms, - 10, - "The max waiting time for replica loading to complete."); +DSN_DEFINE_uint64(replication, load_replica_max_wait_time_ms, 10, kLoadReplicaMaxWaitTimeMsDesc); +DSN_TAG_VARIABLE(load_replica_max_wait_time_ms, FT_MUTABLE); +DSN_DEFINE_validator(load_replica_max_wait_time_ms, + [](uint64_t value) -> bool { return value > 0; }); DSN_DEFINE_bool(replication, deny_client_on_start, @@ -401,6 +408,40 @@ namespace dsn { namespace replication { bool replica_stub::s_not_exit_on_log_failure = false; +namespace { + +void register_flags_ctrl_command() +{ + static std::once_flag flag; + static std::vector> cmds; + std::call_once(flag, []() mutable { + cmds.emplace_back(dsn::command_manager::instance().register_int_command( + FLAGS_max_replicas_on_load_for_each_disk, + FLAGS_max_replicas_on_load_for_each_disk, + "replica.max-replicas-on-load-for-each-disk", + kMaxReplicasOnLoadForEachDiskDesc)); + + cmds.emplace_back(dsn::command_manager::instance().register_int_command( + FLAGS_load_replica_max_wait_time_ms, + FLAGS_load_replica_max_wait_time_ms, + "replica.load-replica-max-wait-time-ms", + kLoadReplicaMaxWaitTimeMsDesc)); + + cmds.emplace_back(dsn::command_manager::instance().register_bool_command( + FLAGS_empty_write_disabled, + "replica.disable-empty-write", + "whether to disable empty writes")); + + cmds.emplace_back(::dsn::command_manager::instance().register_int_command( + FLAGS_max_concurrent_bulk_load_downloading_count, + FLAGS_max_concurrent_bulk_load_downloading_count, + "replica.max-concurrent-bulk-load-downloading-count", + kMaxConcurrentBulkLoadDownloadingCountDesc)); + }); +} + +} // anonymous namespace + replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/, bool is_long_subscriber /* = true*/) : serverlet("replica_stub"), @@ -452,6 +493,8 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/, _is_long_subscriber = is_long_subscriber; _failure_detector = nullptr; _state = NS_Disconnected; + + register_flags_ctrl_command(); } replica_stub::~replica_stub(void) { close(); } @@ -531,14 +574,17 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) { const auto &disks = get_all_disk_dirs(); - std::vector dir_indexes(disks.size(), 0); + // The index of currently loaded replica dir for each disk. Once current replica + std::vector replica_dir_indexes(disks.size(), 0); struct replica_dir_loader { - size_t dir_index; - std::string dir_path; - task_ptr load_task; + size_t replica_dir_index; + std::string replica_dir_path; + task_ptr load_replica_task; }; + + // std::vector> load_disk_queues(disks.size()); utils::ex_lock reps_lock; @@ -561,8 +607,8 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support. const auto &dirs = disks[disk_index].replica_dirs; - auto &dir_index = dir_indexes[disk_index]; - if (dir_index >= dirs.size()) { + auto &replica_dir_index = replica_dir_indexes[disk_index]; + if (replica_dir_index >= dirs.size()) { // All of the replicas for the disk `disks[disk_index]` have begun to be loaded, // thus just skip. ++finished_disks; @@ -573,7 +619,7 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) auto &load_disk_queue = load_disk_queues[disk_index]; if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { // Loading replicas should be throttled in case that disk IO is saturated. - if (!load_disk_queue.front().load_task->wait( + if (!load_disk_queue.front().load_replica_task->wait( static_cast(FLAGS_load_replica_max_wait_time_ms))) { // There might be too many replicas that are being loaded which lead to // slow disk IO. @@ -582,14 +628,14 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) "={}, tag={}, path={}), skip dir(index={}, path={}), turn to " "next disk", FLAGS_load_replica_max_wait_time_ms, - load_disk_queue.front().dir_index, - load_disk_queue.front().dir_path, + load_disk_queue.front().replica_dir_index, + load_disk_queue.front().replica_dir_path, load_disk_queue.size(), disk_index, dn->tag, dn->full_dir, - dir_index, - dirs[dir_index]); + replica_dir_index, + dirs[replica_dir_index]); continue; } @@ -597,22 +643,23 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) load_disk_queue.pop(); } - if (dsn::replication::is_data_dir_invalid(dirs[dir_index])) { - LOG_WARNING("ignore dir(index={}, path={})", dir_index, dirs[dir_index]); - ++dir_index; + if (dsn::replication::is_data_dir_invalid(dirs[replica_dir_index])) { + LOG_WARNING( + "ignore dir(index={}, path={})", replica_dir_index, dirs[replica_dir_index]); + ++replica_dir_index; continue; } LOG_DEBUG("ready to load dir(index={}, path={}) for disk(index={}, tag={}, path={})", - dir_index, - dirs[dir_index], + replica_dir_index, + dirs[replica_dir_index], disk_index, dn->tag, dn->full_dir); load_disk_queue.push(replica_dir_loader{ - dir_index, - dirs[dir_index], + replica_dir_index, + dirs[replica_dir_index], tasking::create_task( // Ensure that the thread pool is non-partitioned. LPC_REPLICATION_INIT_LOAD, @@ -624,13 +671,13 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) &replica_stub::load_replica), this, dn, - dirs[dir_index], + dirs[replica_dir_index], std::ref(reps_lock), std::ref(reps)))}); - load_disk_queue.back().load_task->enqueue(); + load_disk_queue.back().load_replica_task->enqueue(); - ++dir_index; + ++replica_dir_index; } if (finished_disks >= disks.size()) { @@ -642,7 +689,7 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) // All loading tasks have been in the queue. Just wait all tasks to be finished. for (auto &load_disk_queue : load_disk_queues) { while (!load_disk_queue.empty()) { - CHECK_TRUE(load_disk_queue.front().load_task->wait()); + CHECK_TRUE(load_disk_queue.front().load_replica_task->wait()); load_disk_queue.pop(); } } @@ -2593,11 +2640,6 @@ void replica_stub::register_ctrl_command() }); })); - _cmds.emplace_back(::dsn::command_manager::instance().register_bool_command( - FLAGS_empty_write_disabled, - "replica.disable-empty-write", - "whether to disable empty writes")); - #ifdef DSN_ENABLE_GPERF _cmds.emplace_back(::dsn::command_manager::instance().register_bool_command( _release_tcmalloc_memory, @@ -2632,21 +2674,18 @@ void replica_stub::register_ctrl_command() #elif defined(DSN_USE_JEMALLOC) register_jemalloc_ctrl_command(); #endif - _cmds.emplace_back(::dsn::command_manager::instance().register_int_command( - FLAGS_max_concurrent_bulk_load_downloading_count, - FLAGS_max_concurrent_bulk_load_downloading_count, - "replica.max-concurrent-bulk-load-downloading-count", - kMaxConcurrentBulkLoadDownloadingCountDesc)); }); } std::string -replica_stub::exec_command_on_replica(const std::vector &args, +replica_stub::exec_command_on_replica(const std::vector &arg_str_list, bool allow_empty_args, std::function func) { - if (args.empty() && !allow_empty_args) { - return std::string("invalid arguments"); + static const std::string kInvalidArguments("invalid arguments"); + + if (arg_str_list.empty() && !allow_empty_args) { + return kInvalidArguments; } replica_map_by_gpid rs; @@ -2657,17 +2696,19 @@ replica_stub::exec_command_on_replica(const std::vector &args, std::set required_ids; replica_map_by_gpid choosed_rs; - if (!args.empty()) { - for (int i = 0; i < args.size(); i++) { - std::vector arg_strs; - utils::split_args(args[i].c_str(), arg_strs, ','); - if (arg_strs.empty()) { - return std::string("invalid arguments"); + if (!arg_str_list.empty()) { + for (const auto &arg_str : arg_str_list) { + std::vector args; + utils::split_args(arg_str.c_str(), args, ','); + if (args.empty()) { + return kInvalidArguments; } - for (const std::string &arg : arg_strs) { - if (arg.empty()) + for (const std::string &arg : args) { + if (arg.empty()) { continue; + } + gpid id; int pid; if (id.parse_from(arg.c_str())) { @@ -2686,7 +2727,7 @@ replica_stub::exec_command_on_replica(const std::vector &args, } } } else { - return std::string("invalid arguments"); + return kInvalidArguments; } } } @@ -2706,8 +2747,10 @@ replica_stub::exec_command_on_replica(const std::vector &args, [rep, &func, &results_lock, &results]() { partition_status::type status = rep->status(); if (status != partition_status::PS_PRIMARY && - status != partition_status::PS_SECONDARY) + status != partition_status::PS_SECONDARY) { return; + } + std::string result = func(rep); ::dsn::zauto_lock l(results_lock); auto &value = results[rep->get_gpid()]; diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index c0bb7f43b6..a6cc9bd917 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -118,7 +118,8 @@ typedef rpc_holder add_new_disk_rpc namespace test { class test_checker; -} +} // namespace test + class cold_backup_context; class replica_split_manager; diff --git a/src/utils/command_manager.h b/src/utils/command_manager.h index b971522d01..828727bcb1 100644 --- a/src/utils/command_manager.h +++ b/src/utils/command_manager.h @@ -65,14 +65,12 @@ class command_manager : public ::dsn::utils::singleton // 'validator' is used to validate the new value. // The value is reset to 'default_value' if passing "DEFAULT" argument. template - WARN_UNUSED_RESULT std::unique_ptr register_int_command( - T &value, - T default_value, - const std::string &command, - const std::string &help, - std::function validator = [](int64_t new_value) -> bool { - return new_value >= 0; - }) + WARN_UNUSED_RESULT std::unique_ptr + register_int_command(T &value, + T default_value, + const std::string &command, + const std::string &help, + std::function::type)> validator) { return register_single_command( command, @@ -83,6 +81,19 @@ class command_manager : public ::dsn::utils::singleton }); } + template + WARN_UNUSED_RESULT std::unique_ptr register_int_command( + T &value, T default_value, const std::string &command, const std::string &help) + { + return register_int_command(value, + default_value, + command, + help, + [](typename std::remove_reference::type new_value) -> bool { + return new_value >= 0; + }); + } + // Register a single 'command' with the 'help' description, its arguments are described in // 'args'. std::unique_ptr @@ -133,11 +144,12 @@ class command_manager : public ::dsn::utils::singleton set_bool(bool &value, const std::string &name, const std::vector &args); template - static std::string set_int(T &value, - T default_value, - const std::string &name, - const std::vector &args, - const std::function &validator) + static std::string + set_int(T &value, + T default_value, + const std::string &name, + const std::vector &args, + const std::function::type)> &validator) { nlohmann::json msg; msg["error"] = "ok"; @@ -164,8 +176,7 @@ class command_manager : public ::dsn::utils::singleton // Invalid argument. T new_value = 0; - if (!internal::buf2signed(args[0], new_value) || - !validator(static_cast(new_value))) { + if (!buf2numeric(args[0], new_value) || !validator(new_value)) { msg["error"] = fmt::format("ERR: invalid argument '{}', the value is not acceptable", args[0]); return msg.dump(2); From 2fd42b9f341908a9cfd312508c3356b5e2d8cbe8 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Thu, 19 Dec 2024 11:13:56 +0800 Subject: [PATCH 36/43] fix clang-tidy --- src/meta/meta_service.cpp | 1 + src/replica/replica_stub.cpp | 92 +++++++++++++++++++----------------- src/replica/replica_stub.h | 4 +- src/server/result_writer.cpp | 1 - src/utils/command_manager.h | 1 + 5 files changed, 53 insertions(+), 46 deletions(-) diff --git a/src/meta/meta_service.cpp b/src/meta/meta_service.cpp index bd9736304b..6ec19c007d 100644 --- a/src/meta/meta_service.cpp +++ b/src/meta/meta_service.cpp @@ -29,6 +29,7 @@ #include #include // for std::remove_if #include +#include #include #include #include diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index a01105db84..3f2428f9ad 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -29,6 +29,7 @@ // IWYU pragma: no_include #include #include +#include #include #include #include @@ -413,30 +414,33 @@ namespace { void register_flags_ctrl_command() { static std::once_flag flag; - static std::vector> cmds; std::call_once(flag, []() mutable { - cmds.emplace_back(dsn::command_manager::instance().register_int_command( - FLAGS_max_replicas_on_load_for_each_disk, - FLAGS_max_replicas_on_load_for_each_disk, - "replica.max-replicas-on-load-for-each-disk", - kMaxReplicasOnLoadForEachDiskDesc)); - - cmds.emplace_back(dsn::command_manager::instance().register_int_command( - FLAGS_load_replica_max_wait_time_ms, - FLAGS_load_replica_max_wait_time_ms, - "replica.load-replica-max-wait-time-ms", - kLoadReplicaMaxWaitTimeMsDesc)); - - cmds.emplace_back(dsn::command_manager::instance().register_bool_command( - FLAGS_empty_write_disabled, - "replica.disable-empty-write", - "whether to disable empty writes")); - - cmds.emplace_back(::dsn::command_manager::instance().register_int_command( - FLAGS_max_concurrent_bulk_load_downloading_count, - FLAGS_max_concurrent_bulk_load_downloading_count, - "replica.max-concurrent-bulk-load-downloading-count", - kMaxConcurrentBulkLoadDownloadingCountDesc)); + dsn::command_manager::instance().add_global_cmd( + dsn::command_manager::instance().register_int_command( + FLAGS_max_replicas_on_load_for_each_disk, + FLAGS_max_replicas_on_load_for_each_disk, + "replica.max-replicas-on-load-for-each-disk", + kMaxReplicasOnLoadForEachDiskDesc)); + + dsn::command_manager::instance().add_global_cmd( + dsn::command_manager::instance().register_int_command( + FLAGS_load_replica_max_wait_time_ms, + FLAGS_load_replica_max_wait_time_ms, + "replica.load-replica-max-wait-time-ms", + kLoadReplicaMaxWaitTimeMsDesc)); + + dsn::command_manager::instance().add_global_cmd( + dsn::command_manager::instance().register_bool_command( + FLAGS_empty_write_disabled, + "replica.disable-empty-write", + "whether to disable empty writes")); + + dsn::command_manager::instance().add_global_cmd( + dsn::command_manager::instance().register_int_command( + FLAGS_max_concurrent_bulk_load_downloading_count, + FLAGS_max_concurrent_bulk_load_downloading_count, + "replica.max-concurrent-bulk-load-downloading-count", + kMaxConcurrentBulkLoadDownloadingCountDesc)); }); } @@ -445,6 +449,9 @@ void register_flags_ctrl_command() replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/, bool is_long_subscriber /* = true*/) : serverlet("replica_stub"), + _state(NS_Disconnected), + _replica_state_subscriber(subscriber), + _is_long_subscriber(is_long_subscriber), _deny_client(false), _verbose_client_log(false), _verbose_commit_log(false), @@ -454,6 +461,9 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/, _bulk_load_downloading_count(0), _manual_emergency_checkpointing_count(0), _is_running(false), +#ifdef DSN_ENABLE_GPERF + _is_releasing_memory(false), +#endif METRIC_VAR_INIT_server(total_replicas), METRIC_VAR_INIT_server(opening_replicas), METRIC_VAR_INIT_server(closing_replicas), @@ -486,18 +496,10 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/, METRIC_VAR_INIT_server(splitting_replicas_async_learn_max_duration_ms), METRIC_VAR_INIT_server(splitting_replicas_max_copy_file_bytes) { -#ifdef DSN_ENABLE_GPERF - _is_releasing_memory = false; -#endif - _replica_state_subscriber = subscriber; - _is_long_subscriber = is_long_subscriber; - _failure_detector = nullptr; - _state = NS_Disconnected; - register_flags_ctrl_command(); } -replica_stub::~replica_stub(void) { close(); } +replica_stub::~replica_stub() { close(); } void replica_stub::initialize(bool clear /* = false*/) { @@ -2680,7 +2682,7 @@ void replica_stub::register_ctrl_command() std::string replica_stub::exec_command_on_replica(const std::vector &arg_str_list, bool allow_empty_args, - std::function func) + std::function func) { static const std::string kInvalidArguments("invalid arguments"); @@ -2710,7 +2712,6 @@ replica_stub::exec_command_on_replica(const std::vector &arg_str_li } gpid id; - int pid; if (id.parse_from(arg.c_str())) { // app_id.partition_index required_ids.insert(id); @@ -2718,17 +2719,22 @@ replica_stub::exec_command_on_replica(const std::vector &arg_str_li if (find != rs.end()) { choosed_rs[id] = find->second; } - } else if (sscanf(arg.c_str(), "%d", &pid) == 1) { - // app_id - for (auto kv : rs) { - id = kv.second->get_gpid(); - if (id.get_app_id() == pid) { - choosed_rs[id] = kv.second; - } - } - } else { + + continue; + } + + int pid = 0; + if (sscanf(arg.c_str(), "%d", &pid) != 1) { return kInvalidArguments; } + + // app_id + for (const auto &[_, rep] : rs) { + id = rep->get_gpid(); + if (id.get_app_id() == pid) { + choosed_rs[id] = rep; + } + } } } } else { diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index a6cc9bd917..c798f39142 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -229,9 +229,9 @@ class replica_stub : public serverlet, public ref_counter // - if allow_empty_args = false, you should specify at least one argument. // each argument should be in format of: // id1,id2... (where id is 'app_id' or 'app_id.partition_id') - std::string exec_command_on_replica(const std::vector &args, + std::string exec_command_on_replica(const std::vector &arg_str_list, bool allow_empty_args, - std::function func); + std::function func); // // partition split diff --git a/src/server/result_writer.cpp b/src/server/result_writer.cpp index df78172e9c..4d3a9160e0 100644 --- a/src/server/result_writer.cpp +++ b/src/server/result_writer.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #include "pegasus/client.h" diff --git a/src/utils/command_manager.h b/src/utils/command_manager.h index 828727bcb1..9f23d24bb7 100644 --- a/src/utils/command_manager.h +++ b/src/utils/command_manager.h @@ -36,6 +36,7 @@ #include #include #include +#include #include #include "utils/fmt_logging.h" From caeafdc7ee817769c47f0cfdf4da855921e34b7f Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Thu, 19 Dec 2024 17:48:32 +0800 Subject: [PATCH 37/43] fix clang-tidy, add timer and comments --- src/replica/replica_stub.cpp | 160 ++++++++++++++++++++++------------- src/replica/replica_stub.h | 8 +- 2 files changed, 106 insertions(+), 62 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 3f2428f9ad..101e91b629 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -31,11 +31,11 @@ #include #include #include -#include -#include #include #include #include +#include +#include #include #include #include @@ -88,7 +88,7 @@ #include "utils/strings.h" #include "utils/synchronize.h" #include "utils/threadpool_spec.h" -#include "utils/time_utils.h" +#include "utils/timer.h" #ifdef DSN_ENABLE_GPERF #include #elif defined(DSN_USE_JEMALLOC) @@ -411,8 +411,11 @@ bool replica_stub::s_not_exit_on_log_failure = false; namespace { +// Register commands that get/set flag configurations. void register_flags_ctrl_command() { + // For the reaonse why using std::call_once please see comments in + // replica_stub::register_ctrl_command() for details. static std::once_flag flag; std::call_once(flag, []() mutable { dsn::command_manager::instance().add_global_cmd( @@ -450,7 +453,7 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/, bool is_long_subscriber /* = true*/) : serverlet("replica_stub"), _state(NS_Disconnected), - _replica_state_subscriber(subscriber), + _replica_state_subscriber(std::move(subscriber)), _is_long_subscriber(is_long_subscriber), _deny_client(false), _verbose_client_log(false), @@ -512,17 +515,17 @@ void replica_stub::initialize(bool clear /* = false*/) std::vector replica_stub::get_all_disk_dirs() const { std::vector disks; - for (const auto &dn : _fs_manager.get_dir_nodes()) { - if (dsn_unlikely(dn->status == disk_status::IO_ERROR)) { + for (const auto &disk_node : _fs_manager.get_dir_nodes()) { + if (dsn_unlikely(disk_node->status == disk_status::IO_ERROR)) { // Skip disks with IO errors. continue; } std::vector sub_dirs; - CHECK(dsn::utils::filesystem::get_subdirectories(dn->full_dir, sub_dirs, false), + CHECK(dsn::utils::filesystem::get_subdirectories(disk_node->full_dir, sub_dirs, false), "failed to get sub_directories in {}", - dn->full_dir); - disks.push_back(disk_replicas_info{dn.get(), std::move(sub_dirs)}); + disk_node->full_dir); + disks.push_back(disk_replicas_info{disk_node.get(), std::move(sub_dirs)}); } return disks; @@ -530,12 +533,16 @@ std::vector replica_stub::get_all_disk_dirs() // TaskCode: LPC_REPLICATION_INIT_LOAD // ThreadPool: THREAD_POOL_LOCAL_APP -void replica_stub::load_replica(dir_node *dn, +void replica_stub::load_replica(dir_node *disk_node, const std::string &replica_dir, + const size_t total_dir_count, utils::ex_lock &reps_lock, - replica_map_by_gpid &reps) + replica_map_by_gpid &reps, + std::atomic &finished_dir_count) { - LOG_INFO("loading replica: tag={}, replica_dir={}", dn->tag, replica_dir); + SCOPED_LOG_TIMING(INFO, "on loading {}:{}", disk_node->tag, replica_dir); + + LOG_INFO("loading replica: replica_dir={}:{}", disk_node->tag, replica_dir); const auto *const worker = task::get_current_worker2(); if (worker != nullptr) { @@ -545,17 +552,24 @@ void replica_stub::load_replica(dir_node *dn, "among multiple threads"); } - auto rep = load_replica(dn, replica_dir); + auto rep = load_replica(disk_node, replica_dir); if (rep == nullptr) { + LOG_INFO("load replica failed: replica_dir={}:{}, progress={}/{}", + disk_node->tag, + replica_dir, + ++finished_dir_count, + total_dir_count); return; } - LOG_INFO("{}@{}: load replica successfully, tag={}, replica_dir={}, last_durable_decree={}, " - "last_committed_decree={}, last_prepared_decree={}", + LOG_INFO("{}@{}: load replica successfully, replica_dir={}:{}, progress={}/{}, " + "last_durable_decree={}, last_committed_decree={}, last_prepared_decree={}", rep->get_gpid(), dsn_primary_host_port(), - dn->tag, + disk_node->tag, replica_dir, + ++finished_dir_count, + total_dir_count, rep->last_durable_decree(), rep->last_committed_decree(), rep->last_prepared_decree()); @@ -574,9 +588,12 @@ void replica_stub::load_replica(dir_node *dn, void replica_stub::load_replicas(replica_map_by_gpid &reps) { + SCOPED_LOG_TIMING(INFO, "on loading replicas"); + const auto &disks = get_all_disk_dirs(); - // The index of currently loaded replica dir for each disk. Once current replica + // The max index of dirs that are currently being loaded for each disk. The dirs with + // higher indexes have not begun to be loaded (namely pushed into the queue). std::vector replica_dir_indexes(disks.size(), 0); struct replica_dir_loader @@ -586,9 +603,19 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) task_ptr load_replica_task; }; - // + // Each queue would cache the tasks that loading dirs for each disk. Once the task is + // found finished (namely a dir has been loaded successfully), it would be popped from + // the queue. std::vector> load_disk_queues(disks.size()); + // TODO(wangdan): calculate the number of successful or failed loading of replica dirs, + // and the number for each reason if failed. + std::vector> finished_replica_dirs(disks.size()); + for (auto &count : finished_replica_dirs) { + count.store(0); + } + + // The lock for operations on the loaded replicas as output. utils::ex_lock reps_lock; while (true) { @@ -602,80 +629,96 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) // both variables until clang has been upgraded to version 16 which could support // that well: // - // const auto &[dn, dirs] = disks[disk_index]; + // const auto &[disk_node, replica_dirs] = disks[disk_index]; // // For the docs of clang 16 please see: // // https://releases.llvm.org/16.0.0/tools/clang/docs/ReleaseNotes.html#c-20-feature-support. - const auto &dirs = disks[disk_index].replica_dirs; + const auto &replica_dirs = disks[disk_index].replica_dirs; auto &replica_dir_index = replica_dir_indexes[disk_index]; - if (replica_dir_index >= dirs.size()) { + if (replica_dir_index >= replica_dirs.size()) { // All of the replicas for the disk `disks[disk_index]` have begun to be loaded, // thus just skip. ++finished_disks; continue; } - const auto &dn = disks[disk_index].disk_node; + const auto &disk_node = disks[disk_index].disk_node; auto &load_disk_queue = load_disk_queues[disk_index]; if (load_disk_queue.size() >= FLAGS_max_replicas_on_load_for_each_disk) { // Loading replicas should be throttled in case that disk IO is saturated. if (!load_disk_queue.front().load_replica_task->wait( static_cast(FLAGS_load_replica_max_wait_time_ms))) { // There might be too many replicas that are being loaded which lead to - // slow disk IO. - LOG_WARNING("after {} ms, loading dir(index={}, path={}) is still not " - "finished, there are {} replicas being loaded for disk(index" - "={}, tag={}, path={}), skip dir(index={}, path={}), turn to " - "next disk", + // slow disk IO, thus turn to load replicas of next disk, and try to load + // dir `replica_dir_index` of this disk in the next round. + LOG_WARNING("after {} ms, loading dir({}, {}/{}) is still not finished, " + "there are {} replicas being loaded for disk({}:{}, {}/{}), " + "now turn to next disk, and will begin to load dir({}, {}/{}) " + "soon", FLAGS_load_replica_max_wait_time_ms, - load_disk_queue.front().replica_dir_index, load_disk_queue.front().replica_dir_path, + load_disk_queue.front().replica_dir_index, + replica_dirs.size(), load_disk_queue.size(), + disk_node->tag, + disk_node->full_dir, disk_index, - dn->tag, - dn->full_dir, + disks.size(), + replica_dirs[replica_dir_index], replica_dir_index, - dirs[replica_dir_index]); + replica_dirs.size()); continue; } - // Continue to load a replica since we are within the limit now. + // Now the queue size is within the limit again, continue to load a new replica dir. load_disk_queue.pop(); } - if (dsn::replication::is_data_dir_invalid(dirs[replica_dir_index])) { - LOG_WARNING( - "ignore dir(index={}, path={})", replica_dir_index, dirs[replica_dir_index]); + if (dsn::replication::is_data_dir_invalid(replica_dirs[replica_dir_index])) { + LOG_WARNING("ignore dir({}, {}/{}) for disk({}:{}, {}/{})", + replica_dirs[replica_dir_index], + replica_dir_index, + replica_dirs.size(), + disk_node->tag, + disk_node->full_dir, + disk_index, + disks.size()); ++replica_dir_index; continue; } - LOG_DEBUG("ready to load dir(index={}, path={}) for disk(index={}, tag={}, path={})", + LOG_DEBUG("ready to load dir({}, {}/{}) for disk({}:{}, {}/{})", + replica_dirs[replica_dir_index], replica_dir_index, - dirs[replica_dir_index], + replica_dirs.size(), + disk_node->tag, + disk_node->full_dir, disk_index, - dn->tag, - dn->full_dir); + disks.size()); load_disk_queue.push(replica_dir_loader{ replica_dir_index, - dirs[replica_dir_index], + replica_dirs[replica_dir_index], tasking::create_task( // Ensure that the thread pool is non-partitioned. LPC_REPLICATION_INIT_LOAD, &_tracker, std::bind(static_cast( + replica_map_by_gpid &, + std::atomic &)>( &replica_stub::load_replica), this, - dn, - dirs[replica_dir_index], + disk_node, + replica_dirs[replica_dir_index], + replica_dirs.size(), std::ref(reps_lock), - std::ref(reps)))}); + std::ref(reps), + std::ref(finished_replica_dirs[disk_index])))}); load_disk_queue.back().load_replica_task->enqueue(); @@ -783,13 +826,9 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f LOG_INFO("start to load replicas"); replica_map_by_gpid reps; - - utils::chronograph chrono; load_replicas(reps); - LOG_INFO("load replicas succeed, replica_count = {}, time_used = {} ms", - reps.size(), - chrono.duration_ms()); + LOG_INFO("load replicas succeed, replica_count = {}", reps.size()); bool is_log_complete = true; for (auto it = reps.begin(); it != reps.end(); ++it) { @@ -2278,7 +2317,7 @@ bool replica_stub::validate_replica_dir(const std::string &dir, return true; } -replica_ptr replica_stub::load_replica(dir_node *dn, const std::string &replica_dir) +replica_ptr replica_stub::load_replica(dir_node *disk_node, const std::string &replica_dir) { FAIL_POINT_INJECT_F("mock_replica_load", [&](std::string_view) -> replica * { return nullptr; }); @@ -2292,14 +2331,14 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const std::string &replica_ } // The replica's directory must exist when creating a replica. - CHECK_EQ(dn->replica_dir(ai.app_type, pid), replica_dir); + CHECK_EQ(disk_node->replica_dir(ai.app_type, pid), replica_dir); - auto *rep = new replica(this, pid, ai, dn, false); + auto *rep = new replica(this, pid, ai, disk_node, false); const auto err = rep->initialize_on_load(); if (err != ERR_OK) { LOG_ERROR("{}: load replica failed, tag={}, replica_dir={}, err={}", rep->name(), - dn->tag, + disk_node->tag, replica_dir, err); delete rep; @@ -2315,7 +2354,10 @@ replica_ptr replica_stub::load_replica(dir_node *dn, const std::string &replica_ return nullptr; } - LOG_INFO("{}: load replica succeed, tag={}, replica_dir={}", rep->name(), dn->tag, replica_dir); + LOG_INFO("{}: load replica succeed, tag={}, replica_dir={}", + rep->name(), + disk_node->tag, + replica_dir); return rep; } @@ -2713,7 +2755,7 @@ replica_stub::exec_command_on_replica(const std::vector &arg_str_li gpid id; if (id.parse_from(arg.c_str())) { - // app_id.partition_index + // Format: app_id.partition_index required_ids.insert(id); auto find = rs.find(id); if (find != rs.end()) { @@ -2723,15 +2765,15 @@ replica_stub::exec_command_on_replica(const std::vector &arg_str_li continue; } - int pid = 0; - if (sscanf(arg.c_str(), "%d", &pid) != 1) { + // Must be app_id. + int32_t app_id = 0; + if (!buf2int32(arg, app_id)) { return kInvalidArguments; } - // app_id for (const auto &[_, rep] : rs) { id = rep->get_gpid(); - if (id.get_app_id() == pid) { + if (id.get_app_id() == app_id) { choosed_rs[id] = rep; } } diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index c798f39142..ed6702fa04 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -394,16 +394,18 @@ class replica_stub : public serverlet, public ref_counter // Load an existing replica which is located in `dn` with `replica_dir`. Usually each // different `dn` represents a unique disk. `replica_dir` is the absolute path of the // directory for a replica. - virtual replica_ptr load_replica(dir_node *dn, const std::string &replica_dir); + virtual replica_ptr load_replica(dir_node *disk_node, const std::string &replica_dir); using replica_map_by_gpid = std::unordered_map; // The same as the above `load_replica` function, except that this function is to load // each replica to `reps` with protection from `reps_lock`. - void load_replica(dir_node *dn, + void load_replica(dir_node *disk_node, const std::string &replica_dir, + const size_t total_dir_count, utils::ex_lock &reps_lock, - replica_map_by_gpid &reps); + replica_map_by_gpid &reps, + std::atomic &finished_dir_count); // Load all replicas simultaneously from all disks to `reps`. void load_replicas(replica_map_by_gpid &reps); From 61912bb6f03eb15c422ac5ccbda89c7c23d709f9 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Thu, 19 Dec 2024 18:01:49 +0800 Subject: [PATCH 38/43] fix clang-tidy --- src/replica/replica_stub.cpp | 4 ++-- src/replica/replica_stub.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 101e91b629..718f5fa686 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -535,7 +535,7 @@ std::vector replica_stub::get_all_disk_dirs() // ThreadPool: THREAD_POOL_LOCAL_APP void replica_stub::load_replica(dir_node *disk_node, const std::string &replica_dir, - const size_t total_dir_count, + size_t total_dir_count, utils::ex_lock &reps_lock, replica_map_by_gpid &reps, std::atomic &finished_dir_count) @@ -707,7 +707,7 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) &_tracker, std::bind(static_cast &)>( diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index ed6702fa04..26a66e51b8 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -402,7 +402,7 @@ class replica_stub : public serverlet, public ref_counter // each replica to `reps` with protection from `reps_lock`. void load_replica(dir_node *disk_node, const std::string &replica_dir, - const size_t total_dir_count, + size_t total_dir_count, utils::ex_lock &reps_lock, replica_map_by_gpid &reps, std::atomic &finished_dir_count); From f0835d602db5b96a14e7c5c2e964f1fdbe7828aa Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Thu, 19 Dec 2024 22:15:59 +0800 Subject: [PATCH 39/43] fix IWYU --- src/replica/replica_stub.cpp | 1 + src/replica/replica_stub.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 718f5fa686..e47c6b0d9b 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -75,6 +75,7 @@ #include "task/task.h" #include "task/task_engine.h" #include "task/task_worker.h" +#include "utils/api_utilities.h" #include "utils/command_manager.h" #include "utils/env.h" #include "utils/errors.h" diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index 26a66e51b8..bb2c44a0be 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -28,6 +28,7 @@ #include #include +#include #include #include #include From b140d5495f7dfdd641f3560b98a191f74fa08619 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 20 Dec 2024 12:06:22 +0800 Subject: [PATCH 40/43] refactor and add comments --- src/replica/replica_stub.cpp | 45 ++++++++++++++++++++---------------- src/replica/replica_stub.h | 15 +++++++----- 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index e47c6b0d9b..a51ea4d7b6 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -34,7 +34,6 @@ #include #include #include -#include #include #include #include @@ -2228,7 +2227,7 @@ replica *replica_stub::new_replica(gpid gpid, bool restore_if_necessary, bool is_duplication_follower) { - return new_replica(gpid, app, restore_if_necessary, is_duplication_follower, ""); + return new_replica(gpid, app, restore_if_necessary, is_duplication_follower, {}); } /*static*/ std::string replica_stub::get_replica_dir_name(const std::string &dir) @@ -3017,27 +3016,33 @@ replica_ptr replica_stub::create_child_replica_if_not_found(gpid child_pid, }); zauto_write_lock l(_replicas_lock); - auto it = _replicas.find(child_pid); + + const auto it = _replicas.find(child_pid); if (it != _replicas.end()) { return it->second; - } else { - if (_opening_replicas.find(child_pid) != _opening_replicas.end()) { - LOG_WARNING("failed create child replica({}) because it is under open", child_pid); - return nullptr; - } else if (_closing_replicas.find(child_pid) != _closing_replicas.end()) { - LOG_WARNING("failed create child replica({}) because it is under close", child_pid); - return nullptr; - } else { - replica *rep = new_replica(child_pid, *app, false, false, parent_dir); - if (rep != nullptr) { - auto pr = _replicas.insert(replica_map_by_gpid::value_type(child_pid, rep)); - CHECK(pr.second, "child replica {} has been existed", rep->name()); - METRIC_VAR_INCREMENT(total_replicas); - _closed_replicas.erase(child_pid); - } - return rep; - } } + + if (_opening_replicas.find(child_pid) != _opening_replicas.end()) { + LOG_WARNING("failed create child replica({}) because it is under open", child_pid); + return nullptr; + } + + if (_closing_replicas.find(child_pid) != _closing_replicas.end()) { + LOG_WARNING("failed create child replica({}) because it is under close", child_pid); + return nullptr; + } + + replica *rep = new_replica(child_pid, *app, false, false, parent_dir); + if (rep == nullptr) { + return nullptr; + } + + const auto pr = _replicas.insert(replica_map_by_gpid::value_type(child_pid, rep)); + CHECK(pr.second, "child replica {} has been existed", rep->name()); + METRIC_VAR_INCREMENT(total_replicas); + _closed_replicas.erase(child_pid); + + return rep; } // ThreadPool: THREAD_POOL_REPLICATION diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index bb2c44a0be..284f178836 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -359,19 +359,22 @@ class replica_stub : public serverlet, public ref_counter gpid id, const std::shared_ptr &req, const std::shared_ptr &req2); - // Create a new replica according to the parameters. - // 'parent_dir' is used in partition split for create_child_replica_dir(). + + // Create a child replica for partition split, with 'parent_dir' specified as the parent + // replica dir used for `create_child_replica_dir()`. replica *new_replica(gpid gpid, const app_info &app, bool restore_if_necessary, bool is_duplication_follower, const std::string &parent_dir); + // Create a new replica, choosing and assigning the best dir for it. replica *new_replica(gpid gpid, const app_info &app, bool restore_if_necessary, bool is_duplication_follower); + // Each disk with its candidate replica dirs, used to load replicas while initializing. struct disk_replicas_info { // `dir_node` for each disk. @@ -384,11 +387,11 @@ class replica_stub : public serverlet, public ref_counter // Get the absolute dirs of all replicas for all disks. std::vector get_all_disk_dirs() const; - // Get the dir name for a replica from a potentially longer path (both absolute and - // relative paths are possible). + // Get the replica dir name from a potentially longer path (`dir` could be an absolute + // or relative path). static std::string get_replica_dir_name(const std::string &dir); - // Parse app id, partition id and app type from the dir name of a replica. + // Parse app id, partition id and app type from the replica dir name. static bool parse_replica_dir_name(const std::string &dir_name, gpid &pid, std::string &app_type); @@ -503,7 +506,7 @@ class replica_stub : public serverlet, public ref_counter using opening_replica_map_by_gpid = std::unordered_map; - // `task_ptr` is the task being closed. + // `task_ptr` is the task closing a replica. using closing_replica_map_by_gpid = std::unordered_map>; From dd324c9aea68f666b4596565e2f6db52098edf16 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Fri, 20 Dec 2024 20:37:45 +0800 Subject: [PATCH 41/43] add comments --- src/replica/replica_stub.cpp | 40 +++++++++++++++++++++++------------- src/replica/replica_stub.h | 2 +- 2 files changed, 27 insertions(+), 15 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index a51ea4d7b6..639f0ab4a8 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -100,22 +100,15 @@ namespace { -const char *kMaxConcurrentBulkLoadDownloadingCountDesc = - "The maximum concurrent bulk load downloading replica count."; - const char *kMaxReplicasOnLoadForEachDiskDesc = "The max number of replicas that are allowed to be loaded simultaneously for each disk dir."; const char *kLoadReplicaMaxWaitTimeMsDesc = "The max waiting time for replica loading to complete."; -} // anonymous namespace +const char *kMaxConcurrentBulkLoadDownloadingCountDesc = + "The maximum concurrent bulk load downloading replica count."; -DSN_DEFINE_int32(replication, - max_concurrent_bulk_load_downloading_count, - 5, - kMaxConcurrentBulkLoadDownloadingCountDesc); -DSN_DEFINE_validator(max_concurrent_bulk_load_downloading_count, - [](int32_t value) -> bool { return value >= 0; }); +} // anonymous namespace METRIC_DEFINE_gauge_int64(server, total_replicas, @@ -290,6 +283,13 @@ DSN_TAG_VARIABLE(load_replica_max_wait_time_ms, FT_MUTABLE); DSN_DEFINE_validator(load_replica_max_wait_time_ms, [](uint64_t value) -> bool { return value > 0; }); +DSN_DEFINE_int32(replication, + max_concurrent_bulk_load_downloading_count, + 5, + kMaxConcurrentBulkLoadDownloadingCountDesc); +DSN_DEFINE_validator(max_concurrent_bulk_load_downloading_count, + [](int32_t value) -> bool { return value >= 0; }); + DSN_DEFINE_bool(replication, deny_client_on_start, false, @@ -540,7 +540,11 @@ void replica_stub::load_replica(dir_node *disk_node, replica_map_by_gpid &reps, std::atomic &finished_dir_count) { - SCOPED_LOG_TIMING(INFO, "on loading {}:{}", disk_node->tag, replica_dir); + // Measure execution time for loading a replica dir. + // + // TODO(wangdan): support decimal milliseconds or microseconds, since loading a small + // replica tends to spend less than 1 milliseconds and show "0ms" in logging. + SCOPED_LOG_TIMING(INFO, "on loading replica dir {}:{}", disk_node->tag, replica_dir); LOG_INFO("loading replica: replica_dir={}:{}", disk_node->tag, replica_dir); @@ -588,14 +592,19 @@ void replica_stub::load_replica(dir_node *disk_node, void replica_stub::load_replicas(replica_map_by_gpid &reps) { + // Measure execution time for loading all replicas from all healthy disks without IO errors. + // + // TODO(wangdan): show both the size of output replicas and execution time on just one + // logging line. SCOPED_LOG_TIMING(INFO, "on loading replicas"); const auto &disks = get_all_disk_dirs(); - // The max index of dirs that are currently being loaded for each disk. The dirs with - // higher indexes have not begun to be loaded (namely pushed into the queue). + // The max index of dirs that are currently being loaded for each disk, which means the dirs + // with higher indexes have not begun to be loaded (namely pushed into the queue). std::vector replica_dir_indexes(disks.size(), 0); + // Each loader is for a replica dir, including its path and loading task. struct replica_dir_loader { size_t replica_dir_index; @@ -608,6 +617,9 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) // the queue. std::vector> load_disk_queues(disks.size()); + // The number of loading replica dirs that have been finished for each disk, used to show + // current progress. + // // TODO(wangdan): calculate the number of successful or failed loading of replica dirs, // and the number for each reason if failed. std::vector> finished_replica_dirs(disks.size()); @@ -639,7 +651,7 @@ void replica_stub::load_replicas(replica_map_by_gpid &reps) auto &replica_dir_index = replica_dir_indexes[disk_index]; if (replica_dir_index >= replica_dirs.size()) { // All of the replicas for the disk `disks[disk_index]` have begun to be loaded, - // thus just skip. + // thus just skip to next disk. ++finished_disks; continue; } diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h index 284f178836..d6162bf8c3 100644 --- a/src/replica/replica_stub.h +++ b/src/replica/replica_stub.h @@ -384,7 +384,7 @@ class replica_stub : public serverlet, public ref_counter std::vector replica_dirs; }; - // Get the absolute dirs of all replicas for all disks. + // Get the absolute dirs of all replicas for all healthy disks without IO errors. std::vector get_all_disk_dirs() const; // Get the replica dir name from a potentially longer path (`dir` could be an absolute From 227a2c4738bda20c12c9c4fb817298711dab34a9 Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Tue, 24 Dec 2024 16:37:28 +0800 Subject: [PATCH 42/43] refactor --- src/replica/replica_stub.cpp | 5 ++- src/replica/test/load_replicas_test.cpp | 44 ++++++++++++++++++------- 2 files changed, 37 insertions(+), 12 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 639f0ab4a8..8ecebd8e4a 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -499,6 +499,9 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/, METRIC_VAR_INIT_server(splitting_replicas_async_learn_max_duration_ms), METRIC_VAR_INIT_server(splitting_replicas_max_copy_file_bytes) { + // Some flags might need to be tuned on the stage of loading replicas (during + // replica_stub::initialize()), thus register their control command just in the + // constructor. register_flags_ctrl_command(); } @@ -522,7 +525,7 @@ std::vector replica_stub::get_all_disk_dirs() } std::vector sub_dirs; - CHECK(dsn::utils::filesystem::get_subdirectories(disk_node->full_dir, sub_dirs, false), + CHECK(utils::filesystem::get_subdirectories(disk_node->full_dir, sub_dirs, false), "failed to get sub_directories in {}", disk_node->full_dir); disks.push_back(disk_replicas_info{disk_node.get(), std::move(sub_dirs)}); diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 2e6c08808a..8de9c661f9 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -68,6 +68,9 @@ class mock_load_replica : public replica_stub tags.push_back(tag); } + // Initialize fs_manager by the tag and dir of each disk. + _fs_manager.initialize(dirs, tags); + // Generate the replicas which are expected after loading. for (const auto &[tag, reps] : replicas_by_tag) { for (const auto &pid : reps) { @@ -75,13 +78,12 @@ class mock_load_replica : public replica_stub } } - // Initialize fs_manager. - _fs_manager.initialize(dirs, tags); - _disk_tags_for_order.clear(); _disk_dirs_for_order.clear(); _disk_replicas_for_order.clear(); _disk_loaded_replicas_for_order.assign(replicas_by_tag.size(), 0); + + // Ensure that the disks are scanned in the order returned by `get_dir_node()`. for (const auto &dn : _fs_manager.get_dir_nodes()) { for (const auto &pid : replicas_by_tag.at(dn->tag)) { _fs_manager.specify_dir_for_new_replica_for_test(dn.get(), "pegasus", pid); @@ -103,10 +105,12 @@ class mock_load_replica : public replica_stub PRESERVE_FLAG(max_replicas_on_load_for_each_disk); FLAGS_max_replicas_on_load_for_each_disk = max_replicas_on_load_for_each_disk; + // Check if all loaded replicas are matched. replica_stub::replica_map_by_gpid actual_loaded_replicas; load_replicas(actual_loaded_replicas); ASSERT_EQ(_expected_loaded_replicas, actual_loaded_replicas); + // Check if all replicas have been loaded. std::set actual_loaded_replica_pids; for (const auto &[pid, _] : actual_loaded_replicas) { ASSERT_TRUE(actual_loaded_replica_pids.insert(pid).second); @@ -139,6 +143,8 @@ class mock_load_replica : public replica_stub // Once the task is `allow_inline`, it would be executed in place immediately rather // than pushed into the queue. Thus we could test the expected order in which the // tasks are pushed into the queue. + + // Find the first disk where there is still some replica that has not been loaded. size_t finished_disks = 0; while (_disk_loaded_replicas_for_order[_disk_index_for_order] >= _disk_replicas_for_order[_disk_index_for_order]) { @@ -153,7 +159,7 @@ class mock_load_replica : public replica_stub // Only check if the processed order of the disk the replica belongs to, rather than // the order of the replica itself, for the reason that the order of the dirs returned - // by the underlying call varies with different systems. + // by the underlying call might vary. ASSERT_EQ(_disk_tags_for_order[_disk_index_for_order], dn->tag); ASSERT_EQ(_disk_dirs_for_order[_disk_index_for_order], dn->full_dir); @@ -189,7 +195,8 @@ class mock_load_replica : public replica_stub std::set _expected_loaded_replica_pids; - // Only for testing the order of the loading tasks. + // The variables with postfix `_for_order` are only for testing the order of the loading + // tasks. size_t _disk_index_for_order{0}; std::vector _disk_tags_for_order; std::vector _disk_dirs_for_order; @@ -205,7 +212,10 @@ class mock_load_replica : public replica_stub struct load_replicas_case { + // Each disk tag => dir of this disk. std::map dirs_by_tag; + + // Each disk tag => replicas (specified by ) on this disk. std::map> replicas_by_tag; }; @@ -214,8 +224,10 @@ class LoadReplicasTest : public testing::TestWithParam public: LoadReplicasTest() { + // Remove all dirs of all disks to prevent each test from being disturbed. _stub.remove_disk_dirs(); + // Use test cases to initialize the replica stub. const auto &load_case = GetParam(); _stub.initialize(load_case.dirs_by_tag, load_case.replicas_by_tag); } @@ -240,6 +252,8 @@ TEST_P(LoadReplicasTest, LoadOrder) { test_load_replicas(true, 256); } TEST_P(LoadReplicasTest, LoadThrottling) { test_load_replicas(false, 5); } +// Generate a test case for loading replicas. Each element in `disk_replicas` is corresponding +// to the number of replicas on a disk. load_replicas_case generate_load_replicas_case(const std::vector &disk_replicas) { std::map dirs_by_tag; @@ -247,9 +261,12 @@ load_replicas_case generate_load_replicas_case(const std::vector &disk_r dirs_by_tag.emplace(fmt::format("data{}", disk_index), fmt::format("disk{}", disk_index)); } - static const int32_t kNumPartitions = 8; - int32_t partition_id = 0; + static const int32_t kNumBitsPartitions = 3; + static const int32_t kNumPartitions = 1 << kNumBitsPartitions; + int32_t app_id = 1; + int32_t partition_id = 0; + std::map> replicas_by_tag; while (true) { @@ -258,18 +275,23 @@ load_replicas_case generate_load_replicas_case(const std::vector &disk_r for (size_t disk_index = 0; disk_index < disk_replicas.size(); ++disk_index) { auto &replica_list = replicas_by_tag[fmt::format("data{}", disk_index)]; if (replica_list.size() >= disk_replicas[disk_index]) { + // All replicas on this disk have been generated, just skip to next disk. ++finished_disks; continue; } + // Generate a replica with current app id and partition index. replica_list.emplace_back(app_id, partition_id); - if (++partition_id >= kNumPartitions) { - partition_id = 0; - ++app_id; - } + + // Once next partition index is found 0, increment app id to turn to next table. + app_id += ((partition_id + 1) & kNumPartitions) >> kNumBitsPartitions; + + // Increment index to turn to next partition. + partition_id = (partition_id + 1) & (kNumPartitions - 1); } if (finished_disks >= disk_replicas.size()) { + // All disks have been done. break; } } From a4ac805283dd07e01fe1fe9c01cbe0c1bdf31b8e Mon Sep 17 00:00:00 2001 From: Dan Wang Date: Tue, 24 Dec 2024 17:06:17 +0800 Subject: [PATCH 43/43] fix clang-tidy --- src/replica/replica_stub.cpp | 7 +++---- src/replica/test/load_replicas_test.cpp | 11 ++++++----- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp index 8ecebd8e4a..aa8336fd26 100644 --- a/src/replica/replica_stub.cpp +++ b/src/replica/replica_stub.cpp @@ -2317,10 +2317,9 @@ bool replica_stub::validate_replica_dir(const std::string &dir, return false; } - // When the online partition split function aborted, the garbage partitions are with pidx in - // the range of [ai.partition_count, 2 * ai.partition_count), which means the partitions with - // pidx >= ai.partition_count are garbage partitions. - if (ai.partition_count <= pid.get_partition_index()) { + if (pid.get_partition_index() >= ai.partition_count) { + // Once the online partition split aborted, the partitions within the range of + // [ai.partition_count, 2 * ai.partition_count) would become garbage. hint_message = fmt::format( "partition[{}], count={}, this replica may be partition split garbage partition, " "ignore it", diff --git a/src/replica/test/load_replicas_test.cpp b/src/replica/test/load_replicas_test.cpp index 8de9c661f9..3315629c93 100644 --- a/src/replica/test/load_replicas_test.cpp +++ b/src/replica/test/load_replicas_test.cpp @@ -261,11 +261,11 @@ load_replicas_case generate_load_replicas_case(const std::vector &disk_r dirs_by_tag.emplace(fmt::format("data{}", disk_index), fmt::format("disk{}", disk_index)); } - static const int32_t kNumBitsPartitions = 3; - static const int32_t kNumPartitions = 1 << kNumBitsPartitions; + static const uint32_t kNumBitsPartitions = 3; + static const uint32_t kNumPartitions = 1U << kNumBitsPartitions; - int32_t app_id = 1; - int32_t partition_id = 0; + uint32_t app_id = 1; + uint32_t partition_id = 0; std::map> replicas_by_tag; @@ -281,7 +281,8 @@ load_replicas_case generate_load_replicas_case(const std::vector &disk_r } // Generate a replica with current app id and partition index. - replica_list.emplace_back(app_id, partition_id); + replica_list.emplace_back(static_cast(app_id), + static_cast(partition_id)); // Once next partition index is found 0, increment app id to turn to next table. app_id += ((partition_id + 1) & kNumPartitions) >> kNumBitsPartitions;